You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@falcon.apache.org by ra...@apache.org on 2014/09/12 00:19:06 UTC

[01/41] git commit: FALCON-589 Add test cases for various feed operations on Hcat feeds contributed by Karishma Gulati

Repository: incubator-falcon
Updated Branches:
  refs/heads/FALCON-585 1f9a27f96 -> 0d3e97d78


FALCON-589 Add test cases for various feed operations on Hcat feeds contributed by Karishma Gulati


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

Branch: refs/heads/FALCON-585
Commit: 5000fbbd687998769cd9aac183a5373ed49ae267
Parents: d5fdb32
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Mon Aug 25 11:30:20 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Mon Aug 25 11:30:20 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   3 +-
 .../regression/hcat/HCatFeedOperationsTest.java | 282 +++++++++++++++++++
 2 files changed, 284 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5000fbbd/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 929a881..1357808 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -5,7 +5,8 @@ Trunk (Unreleased)
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
-
+   FALCON-589 Add test cases for various feed operations on Hcat feeds (Karishma G 
+   via Samarth Gupta)
   IMPROVEMENTS
    FALCON-619 ELExp_FutureAndLatestTest stabilization (Paul Isaychuk via Arpit Gupta)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5000fbbd/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
new file mode 100644
index 0000000..d994780
--- /dev/null
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
@@ -0,0 +1,282 @@
+/**
+ * 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.hcat;
+
+import org.apache.falcon.entity.v0.EntityType;
+import org.apache.falcon.entity.v0.Frequency;
+import org.apache.falcon.entity.v0.cluster.Interfacetype;
+import org.apache.falcon.entity.v0.feed.ActionType;
+import org.apache.falcon.entity.v0.feed.ClusterType;
+import org.apache.falcon.regression.Entities.FeedMerlin;
+import org.apache.falcon.regression.core.bundle.Bundle;
+import org.apache.falcon.regression.core.enumsAndConstants.FeedType;
+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.HCatUtil;
+import org.apache.falcon.regression.core.util.OSUtil;
+import org.apache.falcon.regression.core.util.Util;
+import org.apache.falcon.regression.core.util.InstanceUtil;
+import org.apache.falcon.regression.core.util.XmlUtil;
+import org.apache.falcon.regression.testHelper.BaseTestClass;
+import org.apache.hive.hcatalog.api.HCatClient;
+import org.apache.hive.hcatalog.api.HCatCreateTableDesc;
+import org.apache.hive.hcatalog.common.HCatException;
+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+import org.apache.log4j.Logger;
+import org.apache.oozie.client.Job;
+import org.apache.oozie.client.OozieClient;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.Test;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HCatFeedOperationsTest extends BaseTestClass {
+
+    ColoHelper cluster = servers.get(0);
+    OozieClient clusterOC = serverOC.get(0);
+    HCatClient clusterHC = cluster.getClusterHelper().getHCatClient();
+
+    ColoHelper cluster2 = servers.get(1);
+    OozieClient cluster2OC = serverOC.get(1);
+    HCatClient cluster2HC = cluster2.getClusterHelper().getHCatClient();
+
+    private String dbName = "default";
+    private String tableName = "hcatFeedOperationsTest";
+    private String randomTblName = "randomTable_HcatFeedOperationsTest";
+    private String feed;
+    private String aggregateWorkflowDir = baseHDFSDir + "/HCatFeedOperationsTest/aggregator";
+    private static final Logger LOGGER = Logger.getLogger(HCatFeedOperationsTest.class);
+
+    public void uploadWorkflow() throws Exception {
+        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
+    }
+
+    @BeforeClass(alwaysRun = true)
+    public void createTestData() throws Exception {
+        //create an empty table for feed operations
+        ArrayList<HCatFieldSchema> partitions = new ArrayList<HCatFieldSchema>();
+        partitions.add(HCatUtil.getStringSchema("year", "yearPartition"));
+        createEmptyTable(clusterHC, dbName, tableName, partitions);
+
+        //A random table to test submission of replication feed when table doesn't exist on target
+        createEmptyTable(clusterHC, dbName, randomTblName, partitions);
+
+        //create empty table on target cluster
+        createEmptyTable(cluster2HC, dbName, tableName, new ArrayList<HCatFieldSchema>());
+    }
+
+    @BeforeMethod(alwaysRun = true)
+    public void setUp(Method method) throws Exception {
+        LOGGER.info("test name: " + method.getName());
+        Bundle bundle = BundleUtil.readHCatBundle();
+        bundles[0] = new Bundle(bundle, cluster.getPrefix());
+        bundles[0].generateUniqueBundle();
+        bundles[0].setClusterInterface(Interfacetype.REGISTRY, cluster.getClusterHelper().getHCatEndpoint());
+
+
+        bundles[1] = new Bundle(bundle, cluster2.getPrefix());
+        bundles[1].generateUniqueBundle();
+        bundles[1].setClusterInterface(Interfacetype.REGISTRY, cluster2.getClusterHelper().getHCatEndpoint());
+    }
+
+    @AfterMethod(alwaysRun = true)
+    public void tearDown() throws HCatException {
+        removeBundles();
+    }
+
+    @AfterClass(alwaysRun = true)
+    public void deleteTable() throws HCatException {
+        clusterHC.dropTable(dbName, tableName, true);
+        clusterHC.dropTable(dbName, randomTblName, true);
+        cluster2HC.dropTable(dbName, tableName, true);
+    }
+
+    /**
+     * Submit Hcat feed when Hcat table mentioned in table uri does not exist. Response should reflect failure.
+     *
+     * @throws Exception
+     */
+    @Test(groups = {"singleCluster"})
+    public void submitFeedWhenTableDoesNotExist() throws Exception {
+        Bundle.submitCluster(bundles[1]);
+        feed = bundles[1].getInputFeedFromBundle();
+        FeedMerlin feedObj = new FeedMerlin(feed);
+        feedObj.setTableValue(dbName, randomTblName, FeedType.YEARLY.getHcatPathValue());
+        ServiceResponse response = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feedObj.toString());
+        AssertUtil.assertFailed(response);
+    }
+
+    /**
+     * Submit Hcat feed when Hcat table mentioned in table uri exists. Delete that feed, and re-submit.
+     * All responses should reflect success.
+     *
+     * @throws Exception
+     */
+    @Test(groups = {"singleCluster"})
+    public void submitFeedPostDeletionWhenTableExists() throws Exception {
+        Bundle.submitCluster(bundles[0]);
+        feed = bundles[0].getInputFeedFromBundle();
+        FeedMerlin feedObj = new FeedMerlin(feed);
+        feedObj.setTableValue(dbName, tableName, FeedType.YEARLY.getHcatPathValue());
+        ServiceResponse response = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feedObj.toString());
+        AssertUtil.assertSucceeded(response);
+
+        response = prism.getFeedHelper().delete(Util.URLS.DELETE_URL, feedObj.toString());
+        AssertUtil.assertSucceeded(response);
+
+        response = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feedObj.toString());
+        AssertUtil.assertSucceeded(response);
+    }
+
+    /**
+     * Submit Hcat Replication feed when Hcat table mentioned in table uri does not exist on target. The response is
+     * Partial, with successful with submit/schedule on source.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void submitAndScheduleReplicationFeedWhenTableDoesNotExistOnTarget() throws Exception {
+        Bundle.submitCluster(bundles[0], bundles[1]);
+        final String startDate = "2010-01-01T20:00Z";
+        final String endDate = "2099-01-01T00:00Z";
+        String tableUri = "catalog:" + dbName + ":" + randomTblName + "#year=${YEAR}";
+        bundles[0].setInputFeedPeriodicity(1, Frequency.TimeUnit.hours);
+        bundles[0].setInputFeedValidity(startDate, endDate);
+        bundles[0].setInputFeedTableUri(tableUri);
+
+        feed = bundles[0].getDataSets().get(0);
+        // set cluster 2 as the target.
+        feed = InstanceUtil.setFeedClusterWithTable(feed,
+                XmlUtil.createValidity(startDate, endDate),
+                XmlUtil.createRtention("months(9000)", ActionType.DELETE),
+                Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
+                tableUri);
+
+        AssertUtil.assertPartial(
+                prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
+                        feed)
+        );
+    }
+
+    /**
+     * Submit Hcat Replication feed when Hcat table mentioned in table uri exists on both source and target. The response is
+     * Psucceeded, and a replication co-rdinator should apear on target oozie. The test however does not ensure that
+     * replication goes through.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget() throws Exception {
+        Bundle.submitCluster(bundles[0], bundles[1]);
+        final String startDate = "2010-01-01T20:00Z";
+        final String endDate = "2099-01-01T00:00Z";
+        String tableUri = "catalog:" + dbName + ":" + tableName + "#year=${YEAR}";
+        bundles[0].setInputFeedPeriodicity(1, Frequency.TimeUnit.hours);
+        bundles[0].setInputFeedValidity(startDate, endDate);
+        bundles[0].setInputFeedTableUri(tableUri);
+
+        feed = bundles[0].getDataSets().get(0);
+        // set cluster 2 as the target.
+        feed = InstanceUtil.setFeedClusterWithTable(feed,
+                XmlUtil.createValidity(startDate, endDate),
+                XmlUtil.createRtention("months(9000)", ActionType.DELETE),
+                Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
+                tableUri);
+
+        AssertUtil.assertSucceeded(
+                prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
+                        feed)
+        );
+        Assert.assertEquals(InstanceUtil
+                .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feed),
+                        "REPLICATION"), 1);
+        //This test doesn't wait for replication to succeed.
+    }
+
+    /**
+     * Submit Hcat Replication feed. Suspend the feed, and check that feed was suspended on
+     * both clusters. Now resume feed, and check that status is running on both clusters.
+     * The test however does not ensure that replication goes through.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void suspendAndResumeReplicationFeed() throws Exception {
+
+        submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
+
+        AssertUtil.assertSucceeded(
+                prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL,
+                        feed)
+        );
+
+        //check that feed suspended on both clusters
+        AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
+        AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.SUSPENDED);
+
+        AssertUtil.assertSucceeded(
+                prism.getFeedHelper().resume(Util.URLS.RESUME_URL,
+                        feed)
+        );
+
+        AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
+        AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.RUNNING);
+    }
+
+    /**
+     * Submit Hcat Replication feed. Delete the feed, and check that feed was deleted on
+     * both clusters. The test however does not ensure that replication goes through.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void deleteReplicationFeed() throws Exception {
+        submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
+
+        AssertUtil.assertSucceeded(
+                prism.getFeedHelper().delete(Util.URLS.DELETE_URL,
+                        feed)
+        );
+        AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.KILLED);
+        AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.KILLED);
+    }
+
+
+    public static void createEmptyTable(HCatClient cli, String dbName, String tabName, List<HCatFieldSchema> partitionCols) throws HCatException{
+
+        ArrayList<HCatFieldSchema> cols = new ArrayList<HCatFieldSchema>();
+        cols.add(HCatUtil.getStringSchema("id", "id comment"));
+        HCatCreateTableDesc tableDesc = HCatCreateTableDesc
+                .create(dbName, tabName, cols)
+                .partCols(partitionCols)
+                .fileFormat("textfile")
+                .ifNotExists(true)
+                .isTableExternal(true)
+                .build();
+        cli.createTable(tableDesc);
+    }
+}


[02/41] git commit: FALCON-630 late data rerun for process broken in trunk. Contributed by Shwetha GS

Posted by ra...@apache.org.
FALCON-630 late data rerun for process broken in trunk. Contributed by Shwetha GS


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

Branch: refs/heads/FALCON-585
Commit: 06f52eca7b324b779aa88bdaaa12df618a03fdb2
Parents: 5000fbb
Author: Shwetha GS <sh...@inmobi.com>
Authored: Tue Aug 26 12:17:56 2014 +0530
Committer: Shwetha GS <sh...@inmobi.com>
Committed: Tue Aug 26 12:18:32 2014 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../falcon/workflow/WorkflowExecutionArgs.java  |  8 +++---
 .../workflow/WorkflowExecutionContext.java      |  8 +++---
 .../metadata/MetadataMappingServiceTest.java    |  4 +--
 .../workflow/WorkflowExecutionContextTest.java  |  4 +--
 .../WorkflowJobEndNotificationServiceTest.java  |  4 +--
 .../falcon/messaging/JMSMessageProducer.java    |  8 +++---
 .../falcon/messaging/FeedProducerTest.java      | 12 ++++-----
 .../messaging/JMSMessageConsumerTest.java       |  4 +--
 .../messaging/JMSMessageProducerTest.java       | 24 +++++++++---------
 .../falcon/messaging/ProcessProducerTest.java   | 12 ++++-----
 .../feed/FeedReplicationCoordinatorBuilder.java | 11 +++++----
 .../feed/FeedRetentionCoordinatorBuilder.java   |  4 +--
 .../ProcessExecutionCoordinatorBuilder.java     | 26 +++++++++++---------
 .../src/main/resources/action/post-process.xml  |  2 +-
 oozie/src/main/resources/action/pre-process.xml |  4 +--
 .../feed/OozieFeedWorkflowBuilderTest.java      |  3 +++
 .../OozieProcessWorkflowBuilderTest.java        | 15 +++++++----
 .../workflow/FalconPostProcessingTest.java      | 12 ++++-----
 .../metadata/LineageMetadataResourceTest.java   |  4 +--
 .../apache/falcon/latedata/LateDataHandler.java | 10 +++++---
 .../falcon/rerun/handler/LateRerunConsumer.java | 22 +++++++++--------
 .../apache/falcon/late/LateDataHandlerIT.java   |  4 +--
 23 files changed, 114 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4e3f919..36ca573 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -59,6 +59,8 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-630 late data rerun for process broken in trunk. (Shwetha GS)
+
    FALCON-611 Post process arg status is in 'FAILED' state always
    (Shwetha GS via Suhas Vasu)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionArgs.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionArgs.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionArgs.java
index 92af3e1..514bafe 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionArgs.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionArgs.java
@@ -58,11 +58,13 @@ public enum WorkflowExecutionArgs {
 
     // what inputs
     INPUT_FEED_NAMES("falconInputFeeds", "name of the feeds which are used as inputs", false),
-    INPUT_FEED_PATHS("falconInputPaths", "comma separated input feed instance paths", false),
+    INPUT_FEED_PATHS("falconInPaths", "comma separated input feed instance paths", false),
+    INPUT_NAMES("falconInputNames", "name of the inputs", false),
+    INPUT_STORAGE_TYPES("falconInputFeedStorageTypes", "input storage types", false),
 
     // what outputs
-    FEED_NAMES("feedNames", "name of the feeds which are generated/replicated/deleted"),
-    FEED_INSTANCE_PATHS("feedInstancePaths", "comma separated feed instance paths"),
+    OUTPUT_FEED_NAMES("feedNames", "name of the feeds which are generated/replicated/deleted"),
+    OUTPUT_FEED_PATHS("feedInstancePaths", "comma separated feed instance paths"),
 
     // broker related parameters
     TOPIC_NAME("topicName", "name of the topic to be used to send JMS message", false),

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
index 786e94f..f5bb782 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
@@ -80,8 +80,8 @@ public class WorkflowExecutionContext {
         WorkflowExecutionArgs.NOMINAL_TIME,
         WorkflowExecutionArgs.OPERATION,
 
-        WorkflowExecutionArgs.FEED_NAMES,
-        WorkflowExecutionArgs.FEED_INSTANCE_PATHS,
+        WorkflowExecutionArgs.OUTPUT_FEED_NAMES,
+        WorkflowExecutionArgs.OUTPUT_FEED_PATHS,
 
         WorkflowExecutionArgs.WORKFLOW_ID,
         WorkflowExecutionArgs.WORKFLOW_USER,
@@ -177,7 +177,7 @@ public class WorkflowExecutionContext {
     }
 
     public String getOutputFeedNames() {
-        return getValue(WorkflowExecutionArgs.FEED_NAMES);
+        return getValue(WorkflowExecutionArgs.OUTPUT_FEED_NAMES);
     }
 
     public String[] getOutputFeedNamesList() {
@@ -185,7 +185,7 @@ public class WorkflowExecutionContext {
     }
 
     public String getOutputFeedInstancePaths() {
-        return getValue(WorkflowExecutionArgs.FEED_INSTANCE_PATHS);
+        return getValue(WorkflowExecutionArgs.OUTPUT_FEED_PATHS);
     }
 
     public String[] getOutputFeedInstancePathsList() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
index b51caf8..2b030fd 100644
--- a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
@@ -644,8 +644,8 @@ public class MetadataMappingServiceTest {
             "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), INPUT_FEED_NAMES,
             "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), INPUT_INSTANCE_PATHS,
 
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
 
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), FALCON_USER,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/common/src/test/java/org/apache/falcon/workflow/WorkflowExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/workflow/WorkflowExecutionContextTest.java b/common/src/test/java/org/apache/falcon/workflow/WorkflowExecutionContextTest.java
index e97175e..a45633b 100644
--- a/common/src/test/java/org/apache/falcon/workflow/WorkflowExecutionContextTest.java
+++ b/common/src/test/java/org/apache/falcon/workflow/WorkflowExecutionContextTest.java
@@ -273,8 +273,8 @@ public class WorkflowExecutionContextTest {
             "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), INPUT_FEED_NAMES,
             "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), INPUT_INSTANCE_PATHS,
 
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
 
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), FALCON_USER,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/common/src/test/java/org/apache/falcon/workflow/WorkflowJobEndNotificationServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/workflow/WorkflowJobEndNotificationServiceTest.java b/common/src/test/java/org/apache/falcon/workflow/WorkflowJobEndNotificationServiceTest.java
index 2fe08e4..9a6ad98 100644
--- a/common/src/test/java/org/apache/falcon/workflow/WorkflowJobEndNotificationServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/workflow/WorkflowJobEndNotificationServiceTest.java
@@ -135,8 +135,8 @@ public class WorkflowJobEndNotificationServiceTest implements WorkflowExecutionL
             "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), INPUT_FEED_NAMES,
             "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), INPUT_INSTANCE_PATHS,
 
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
 
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), FALCON_USER,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
index 39d6fab..fc31bab 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
@@ -205,13 +205,13 @@ public class JMSMessageProducer {
 
             // override default values
             if (context.getEntityType().equalsIgnoreCase("PROCESS")) {
-                change(message, WorkflowExecutionArgs.FEED_NAMES, feedNames[i]);
+                change(message, WorkflowExecutionArgs.OUTPUT_FEED_NAMES, feedNames[i]);
             } else {
-                change(message, WorkflowExecutionArgs.FEED_NAMES,
-                        message.get(WorkflowExecutionArgs.FEED_NAMES.getName()));
+                change(message, WorkflowExecutionArgs.OUTPUT_FEED_NAMES,
+                        message.get(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()));
             }
 
-            change(message, WorkflowExecutionArgs.FEED_INSTANCE_PATHS, feedPaths[i]);
+            change(message, WorkflowExecutionArgs.OUTPUT_FEED_PATHS, feedPaths[i]);
             messages.add(message);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
index 1c10be5..c45ea1e 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
@@ -71,8 +71,8 @@ public class FeedProducerTest {
 
         args = new String[] {
             "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), TOPIC_NAME,
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "click-logs",
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "click-logs",
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
             "/click-logs/10/05/05/00/20",
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), "falcon",
@@ -179,7 +179,7 @@ public class FeedProducerTest {
         }
         System.out.println("Consumed: " + m.toString());
         assertMessage(m);
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/20");
 
         for (m = null; m == null;) {
@@ -187,7 +187,7 @@ public class FeedProducerTest {
         }
         System.out.println("Consumed: " + m.toString());
         assertMessage(m);
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/21");
 
         for (m = null; m == null;) {
@@ -195,7 +195,7 @@ public class FeedProducerTest {
         }
         System.out.println("Consumed: " + m.toString());
         assertMessage(m);
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/22");
 
         for (m = null; m == null;) {
@@ -203,7 +203,7 @@ public class FeedProducerTest {
         }
         System.out.println("Consumed: " + m.toString());
         assertMessage(m);
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/23");
 
         connection.close();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
index b1f8271..9a4a6f7 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
@@ -105,9 +105,9 @@ public class JMSMessageConsumerTest {
         message.put(WorkflowExecutionArgs.CLUSTER_NAME.getName(), "cluster1");
         message.put(WorkflowExecutionArgs.ENTITY_NAME.getName(), "process1");
         message.put(WorkflowExecutionArgs.ENTITY_TYPE.getName(), "PROCESS");
-        message.put(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+        message.put(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
                 "/clicks/hour/00/0" + i);
-        message.put(WorkflowExecutionArgs.FEED_NAMES.getName(), "clicks");
+        message.put(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "clicks");
         message.put(WorkflowExecutionArgs.LOG_FILE.getName(), "/logfile");
         message.put(WorkflowExecutionArgs.LOG_DIR.getName(), "/tmp/log");
         message.put(WorkflowExecutionArgs.NOMINAL_TIME.getName(), "2012-10-10-10-10");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
index 34cff77..e4ea22f 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
@@ -72,8 +72,8 @@ public class JMSMessageProducerTest {
         List<String> args = createCommonArgs();
         List<String> newArgs = new ArrayList<String>(Arrays.asList(
                 "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), "agg-coord",
-                "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "click-logs,raw-logs",
-                "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "click-logs,raw-logs",
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
                 "/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20"));
         args.addAll(newArgs);
         List<String[]> messages = new ArrayList<String[]>();
@@ -81,10 +81,10 @@ public class JMSMessageProducerTest {
         testProcessMessageCreator(messages, TOPIC_NAME);
         for (MapMessage m : mapMessages) {
             assertMessage(m);
-            Assert.assertTrue((m.getString(WorkflowExecutionArgs.FEED_NAMES.getName())
+            Assert.assertTrue((m.getString(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName())
                     .equals("click-logs,raw-logs")));
             Assert.assertTrue(m
-                    .getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName())
+                    .getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName())
                     .equals("/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20"));
         }
     }
@@ -94,8 +94,8 @@ public class JMSMessageProducerTest {
         List<String> args = createCommonArgs();
         List<String> newArgs = new ArrayList<String>(Arrays.asList(
                 "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), "agg-coord",
-                "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "null",
-                "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), "null"));
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "null",
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), "null"));
         args.addAll(newArgs);
         List<String[]> messages = new ArrayList<String[]>();
         messages.add(args.toArray(new String[args.size()]));
@@ -103,9 +103,9 @@ public class JMSMessageProducerTest {
         for (MapMessage m : mapMessages) {
             assertMessage(m);
             assertMessage(m);
-            Assert.assertTrue(m.getString(WorkflowExecutionArgs.FEED_NAMES.getName()).equals(
+            Assert.assertTrue(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()).equals(
                     "null"));
-            Assert.assertTrue(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName())
+            Assert.assertTrue(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName())
                     .equals("null"));
         }
     }
@@ -116,8 +116,8 @@ public class JMSMessageProducerTest {
         List<String> args = createCommonArgs();
         List<String> newArgs = new ArrayList<String>(Arrays.asList(
                 "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), "agg-coord",
-                "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "raw-logs",
-                "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "raw-logs",
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
                 "/raw-logs/10/05/05/00/20"));
         args.addAll(newArgs);
         messages.add(args.toArray(new String[args.size()]));
@@ -125,8 +125,8 @@ public class JMSMessageProducerTest {
         args = createCommonArgs();
         newArgs = new ArrayList<String>(Arrays.asList(
                 "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), "agg-coord",
-                "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "click-logs",
-                "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "click-logs",
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
                 "/click-logs/10/05/05/00/20"));
         args.addAll(newArgs);
         messages.add(args.toArray(new String[args.size()]));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
index ccb47df..80c2701 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
@@ -52,8 +52,8 @@ public class ProcessProducerTest {
     public void setup() throws Exception {
         args = new String[] {
             "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), ENTITY_NAME,
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "click-logs,raw-logs",
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "click-logs,raw-logs",
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
             "/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), "falcon",
@@ -133,8 +133,8 @@ public class ProcessProducerTest {
         }
         System.out.println("Consumed: " + m.toString());
         assertMessage(m);
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_NAMES.getName()), "click-logs");
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()), "click-logs");
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                 "/click-logs/10/05/05/00/20");
 
         for (m = null; m == null;) {
@@ -142,8 +142,8 @@ public class ProcessProducerTest {
         }
         System.out.println("Consumed: " + m.toString());
         assertMessage(m);
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_NAMES.getName()), "raw-logs");
-        Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()), "raw-logs");
+        Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                 "/raw-logs/10/05/05/00/20");
         connection.close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
index f0864db..966f90e 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
@@ -261,16 +261,17 @@ public class FeedReplicationCoordinatorBuilder extends OozieCoordinatorBuilder<F
                                              String falconFeedStorageType, Properties props) {
         // todo these pairs are the same but used in different context
         // late data handler - should-record action
-        props.put("falconInputFeeds", entity.getName());
-        props.put("falconInPaths", instancePaths);
+        props.put(WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), entity.getName());
+        props.put(WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), instancePaths);
+        props.put(WorkflowExecutionArgs.INPUT_NAMES.getName(), entity.getName());
 
         // storage type for each corresponding feed - in this case only one feed is involved
         // needed to compute usage based on storage type in LateDataHandler
-        props.put("falconInputFeedStorageTypes", falconFeedStorageType);
+        props.put(WorkflowExecutionArgs.INPUT_STORAGE_TYPES.getName(), falconFeedStorageType);
 
         // falcon post processing
-        props.put(WorkflowExecutionArgs.FEED_NAMES.getName(), entity.getName());
-        props.put(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), "${coord:dataOut('output')}");
+        props.put(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), entity.getName());
+        props.put(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), "${coord:dataOut('output')}");
     }
 
     private void setupHiveConfiguration(Cluster srcCluster, Cluster trgCluster,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedRetentionCoordinatorBuilder.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedRetentionCoordinatorBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedRetentionCoordinatorBuilder.java
index 3c74485..c896d5a 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedRetentionCoordinatorBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedRetentionCoordinatorBuilder.java
@@ -86,8 +86,8 @@ public class FeedRetentionCoordinatorBuilder extends OozieCoordinatorBuilder<Fee
 
         props.put("limit", feedCluster.getRetention().getLimit().toString());
 
-        props.put(WorkflowExecutionArgs.FEED_NAMES.getName(), entity.getName());
-        props.put(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), IGNORE);
+        props.put(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), entity.getName());
+        props.put(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), IGNORE);
 
         props.put("falconInputFeeds", entity.getName());
         props.put("falconInPaths", IGNORE);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/main/java/org/apache/falcon/oozie/process/ProcessExecutionCoordinatorBuilder.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/process/ProcessExecutionCoordinatorBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/process/ProcessExecutionCoordinatorBuilder.java
index a33fa62..1fa6758 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/process/ProcessExecutionCoordinatorBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/process/ProcessExecutionCoordinatorBuilder.java
@@ -152,12 +152,14 @@ public class ProcessExecutionCoordinatorBuilder extends OozieCoordinatorBuilder<
 
     private void initializeInputPaths(Cluster cluster, COORDINATORAPP coord, Properties props) throws FalconException {
         if (entity.getInputs() == null) {
-            props.put("falconInputFeeds", "NONE");
-            props.put("falconInPaths", IGNORE);
+            props.put(WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), "NONE");
+            props.put(WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), IGNORE);
+            props.put(WorkflowExecutionArgs.INPUT_NAMES.getName(), IGNORE);
             return;
         }
 
         List<String> inputFeeds = new ArrayList<String>();
+        List<String> inputNames = new ArrayList<String>();
         List<String> inputPaths = new ArrayList<String>();
         List<String> inputFeedStorageTypes = new ArrayList<String>();
         for (Input input : entity.getInputs().getInputs()) {
@@ -190,21 +192,23 @@ public class ProcessExecutionCoordinatorBuilder extends OozieCoordinatorBuilder<
 
             inputFeeds.add(feed.getName());
             inputPaths.add(inputExpr);
+            inputNames.add(input.getName());
             inputFeedStorageTypes.add(storage.getType().name());
         }
 
-        propagateLateDataProperties(inputFeeds, inputPaths, inputFeedStorageTypes, props);
+        propagateLateDataProperties(inputFeeds, inputNames, inputPaths, inputFeedStorageTypes, props);
     }
 
-    private void propagateLateDataProperties(List<String> inputFeeds, List<String> inputPaths,
+    private void propagateLateDataProperties(List<String> inputFeeds, List<String> inputNames, List<String> inputPaths,
         List<String> inputFeedStorageTypes, Properties props) {
         // populate late data handler - should-record action
-        props.put("falconInputFeeds", StringUtils.join(inputFeeds, '#'));
-        props.put("falconInPaths", StringUtils.join(inputPaths, '#'));
+        props.put(WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), StringUtils.join(inputFeeds, '#'));
+        props.put(WorkflowExecutionArgs.INPUT_NAMES.getName(), StringUtils.join(inputNames, '#'));
+        props.put(WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), StringUtils.join(inputPaths, '#'));
 
         // storage type for each corresponding feed sent as a param to LateDataHandler
         // needed to compute usage based on storage type in LateDataHandler
-        props.put("falconInputFeedStorageTypes", StringUtils.join(inputFeedStorageTypes, '#'));
+        props.put(WorkflowExecutionArgs.INPUT_STORAGE_TYPES.getName(), StringUtils.join(inputFeedStorageTypes, '#'));
     }
 
     private SYNCDATASET createDataSet(Feed feed, Cluster cluster, Storage storage,
@@ -250,8 +254,8 @@ public class ProcessExecutionCoordinatorBuilder extends OozieCoordinatorBuilder<
 
     private void initializeOutputPaths(Cluster cluster, COORDINATORAPP coord, Properties props) throws FalconException {
         if (entity.getOutputs() == null) {
-            props.put(WorkflowExecutionArgs.FEED_NAMES.getName(), "NONE");
-            props.put(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), IGNORE);
+            props.put(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "NONE");
+            props.put(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), IGNORE);
             return;
         }
 
@@ -289,8 +293,8 @@ public class ProcessExecutionCoordinatorBuilder extends OozieCoordinatorBuilder<
         }
 
         // Output feed name and path for parent workflow
-        props.put(WorkflowExecutionArgs.FEED_NAMES.getName(), StringUtils.join(outputFeeds, ','));
-        props.put(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), StringUtils.join(outputPaths, ','));
+        props.put(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), StringUtils.join(outputFeeds, ','));
+        props.put(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), StringUtils.join(outputPaths, ','));
     }
 
     private DATAOUT createDataOut(Output output) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/main/resources/action/post-process.xml
----------------------------------------------------------------------
diff --git a/oozie/src/main/resources/action/post-process.xml b/oozie/src/main/resources/action/post-process.xml
index 2714859..979d4f0 100644
--- a/oozie/src/main/resources/action/post-process.xml
+++ b/oozie/src/main/resources/action/post-process.xml
@@ -80,7 +80,7 @@
         <arg>${wf:user()}</arg>
         <arg>-falconInputFeeds</arg>
         <arg>${falconInputFeeds}</arg>
-        <arg>-falconInputPaths</arg>
+        <arg>-falconInPaths</arg>
         <arg>${falconInPaths}</arg>
         <file>${wf:conf("falcon.libpath")}/activemq-core.jar</file>
         <file>${wf:conf("falcon.libpath")}/geronimo-j2ee-management.jar</file>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/main/resources/action/pre-process.xml
----------------------------------------------------------------------
diff --git a/oozie/src/main/resources/action/pre-process.xml b/oozie/src/main/resources/action/pre-process.xml
index 127ab80..070c42b 100644
--- a/oozie/src/main/resources/action/pre-process.xml
+++ b/oozie/src/main/resources/action/pre-process.xml
@@ -39,8 +39,8 @@
         <arg>${logDir}/latedata/${nominalTime}/${srcClusterName}</arg>
         <arg>-paths</arg>
         <arg>${falconInPaths}</arg>
-        <arg>-falconInputFeeds</arg>
-        <arg>${falconInputFeeds}</arg>
+        <arg>-falconInputNames</arg>
+        <arg>${falconInputNames}</arg>
         <arg>-falconInputFeedStorageTypes</arg>
         <arg>${falconInputFeedStorageTypes}</arg>
         <capture-output/>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java b/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
index e47895f..3c49353 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
@@ -48,6 +48,7 @@ import org.apache.falcon.security.CurrentUser;
 import org.apache.falcon.security.SecurityUtil;
 import org.apache.falcon.util.RuntimeProperties;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.falcon.workflow.WorkflowExecutionArgs;
 import org.apache.falcon.workflow.WorkflowExecutionContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -206,6 +207,7 @@ public class OozieFeedWorkflowBuilderTest extends AbstractTestBase {
 
         // verify the late data params
         Assert.assertEquals(props.get("falconInputFeeds"), feed.getName());
+        Assert.assertEquals(props.get(WorkflowExecutionArgs.INPUT_NAMES.getName()), feed.getName());
         Assert.assertEquals(props.get("falconInPaths"), "${coord:dataIn('input')}");
         Assert.assertEquals(props.get("falconInPaths"), pathsWithPartitions);
         Assert.assertEquals(props.get("falconInputFeedStorageTypes"), Storage.TYPE.FILESYSTEM.name());
@@ -443,6 +445,7 @@ public class OozieFeedWorkflowBuilderTest extends AbstractTestBase {
         // verify the late data params
         Assert.assertEquals(props.get("falconInputFeeds"), tableFeed.getName());
         Assert.assertEquals(props.get("falconInPaths"), "${coord:dataIn('input')}");
+        Assert.assertEquals(props.get(WorkflowExecutionArgs.INPUT_NAMES.getName()), tableFeed.getName());
         Assert.assertEquals(props.get("falconInputFeedStorageTypes"), Storage.TYPE.TABLE.name());
 
         // verify the post processing params

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java b/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
index 4daf5d8..b1d7a8a 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
@@ -188,6 +188,9 @@ public class OozieProcessWorkflowBuilderTest extends AbstractTestBase {
 
         HashMap<String, String> props = getCoordProperties(coord);
         assertEquals(props.get("mapred.job.priority"), "LOW");
+        List<Input> inputs = process.getInputs().getInputs();
+        assertEquals(props.get(WorkflowExecutionArgs.INPUT_NAMES.getName()), inputs.get(0).getName() + "#" + inputs
+            .get(1).getName());
 
         verifyEntityProperties(process, cluster,
                 WorkflowExecutionContext.EntityOperations.GENERATE, props);
@@ -558,6 +561,7 @@ public class OozieProcessWorkflowBuilderTest extends AbstractTestBase {
         Assert.assertEquals(props.get("falconInputFeeds"), process.getInputs().getInputs().get(0).getFeed());
         Assert.assertEquals(props.get("falconInPaths"), "${coord:dataIn('input')}");
         Assert.assertEquals(props.get("falconInputFeedStorageTypes"), Storage.TYPE.TABLE.name());
+        Assert.assertEquals(props.get("falconInputs"), process.getInputs().getInputs().get(0).getName());
 
         // verify the post processing params
         Assert.assertEquals(props.get("feedNames"), process.getOutputs().getOutputs().get(0).getFeed());
@@ -688,10 +692,11 @@ public class OozieProcessWorkflowBuilderTest extends AbstractTestBase {
         verifyBrokerProperties(cluster, props);
 
         String[] expected = {
-            WorkflowExecutionArgs.FEED_NAMES.getName(),
-            WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+            WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(),
+            WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
             WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(),
-            "falconInPaths",
+            WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(),
+            WorkflowExecutionArgs.INPUT_NAMES.getName(),
             WorkflowExecutionArgs.USER_WORKFLOW_NAME.getName(),
             WorkflowExecutionArgs.USER_WORKFLOW_VERSION.getName(),
             WorkflowExecutionArgs.USER_WORKFLOW_ENGINE.getName(),
@@ -729,7 +734,7 @@ public class OozieProcessWorkflowBuilderTest extends AbstractTestBase {
         verifyBrokerProperties(cluster, props);
 
         Assert.assertEquals(props.get(WorkflowExecutionArgs.INPUT_FEED_NAMES.getName()), "clicks");
-        Assert.assertEquals(props.get(WorkflowExecutionArgs.FEED_NAMES.getName()), "NONE");
+        Assert.assertEquals(props.get(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()), "NONE");
     }
 
     @Test
@@ -758,7 +763,7 @@ public class OozieProcessWorkflowBuilderTest extends AbstractTestBase {
                 WorkflowExecutionContext.EntityOperations.GENERATE, props);
         verifyBrokerProperties(cluster, props);
 
-        Assert.assertEquals(props.get(WorkflowExecutionArgs.FEED_NAMES.getName()), "impressions");
+        Assert.assertEquals(props.get(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()), "impressions");
         Assert.assertEquals(props.get(WorkflowExecutionArgs.INPUT_FEED_NAMES.getName()), "NONE");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
index 91559a5..201b682 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
@@ -46,8 +46,8 @@ public class FalconPostProcessingTest {
     public void setup() throws Exception {
         args = new String[]{
             "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), ENTITY_NAME,
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), "out-click-logs,out-raw-logs",
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(),
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "out-click-logs,out-raw-logs",
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
             "/out-click-logs/10/05/05/00/20,/out-raw-logs/10/05/05/00/20",
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), "falcon",
@@ -131,13 +131,13 @@ public class FalconPostProcessingTest {
 
         assertMessage(m);
         if (topic.equals(FALCON_TOPIC_NAME)) {
-            Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_NAMES.getName()),
+            Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()),
                     "out-click-logs,out-raw-logs");
-            Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+            Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                     "/out-click-logs/10/05/05/00/20,/out-raw-logs/10/05/05/00/20");
         } else {
-            Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_NAMES.getName()), "out-click-logs");
-            Assert.assertEquals(m.getString(WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName()),
+            Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName()), "out-click-logs");
+            Assert.assertEquals(m.getString(WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName()),
                     "/out-click-logs/10/05/05/00/20");
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/prism/src/test/java/org/apache/falcon/resource/metadata/LineageMetadataResourceTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/resource/metadata/LineageMetadataResourceTest.java b/prism/src/test/java/org/apache/falcon/resource/metadata/LineageMetadataResourceTest.java
index d9e9f86..9bef7f5 100644
--- a/prism/src/test/java/org/apache/falcon/resource/metadata/LineageMetadataResourceTest.java
+++ b/prism/src/test/java/org/apache/falcon/resource/metadata/LineageMetadataResourceTest.java
@@ -542,8 +542,8 @@ public class LineageMetadataResourceTest {
             "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), INPUT_FEED_NAMES,
             "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), INPUT_INSTANCE_PATHS,
 
-            "-" + WorkflowExecutionArgs.FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
 
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), FALCON_USER,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java b/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
index 75de12e..d854bdd 100644
--- a/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
@@ -26,6 +26,7 @@ import org.apache.falcon.entity.CatalogStorage;
 import org.apache.falcon.entity.FeedHelper;
 import org.apache.falcon.entity.Storage;
 import org.apache.falcon.hadoop.HadoopClientFactory;
+import org.apache.falcon.workflow.WorkflowExecutionArgs;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -72,12 +73,12 @@ public class LateDataHandler extends Configured implements Tool {
         opt.setRequired(true);
         options.addOption(opt);
 
-        opt = new Option("falconInputFeeds", true,
+        opt = new Option(WorkflowExecutionArgs.INPUT_NAMES.getName(), true,
                 "Input feed names, further separated by #");
         opt.setRequired(true);
         options.addOption(opt);
 
-        opt = new Option("falconInputFeedStorageTypes", true,
+        opt = new Option(WorkflowExecutionArgs.INPUT_STORAGE_TYPES.getName(), true,
                 "Feed storage types corresponding to Input feed names, separated by #");
         opt.setRequired(true);
         options.addOption(opt);
@@ -94,9 +95,10 @@ public class LateDataHandler extends Configured implements Tool {
             return 0;
         }
 
-        String[] inputFeeds = getOptionValue(command, "falconInputFeeds").split("#");
+        String[] inputFeeds = getOptionValue(command, WorkflowExecutionArgs.INPUT_NAMES.getName()).split("#");
         String[] pathGroups = pathStr.split("#");
-        String[] inputFeedStorageTypes = getOptionValue(command, "falconInputFeedStorageTypes").split("#");
+        String[] inputFeedStorageTypes =
+            getOptionValue(command, WorkflowExecutionArgs.INPUT_STORAGE_TYPES.getName()).split("#");
 
         Map<String, Long> metrics = computeMetrics(inputFeeds, pathGroups, inputFeedStorageTypes);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
index 16e340e..80a3b83 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
@@ -26,6 +26,7 @@ import org.apache.falcon.hadoop.HadoopClientFactory;
 import org.apache.falcon.latedata.LateDataHandler;
 import org.apache.falcon.rerun.event.LaterunEvent;
 import org.apache.falcon.rerun.queue.DelayedQueue;
+import org.apache.falcon.workflow.WorkflowExecutionArgs;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -87,11 +88,12 @@ public class LateRerunConsumer<T extends LateRerunHandler<DelayedQueue<LaterunEv
         LateDataHandler late = new LateDataHandler();
         Properties properties = handler.getWfEngine().getWorkflowProperties(
                 message.getClusterName(), message.getWfId());
-        String falconInputFeeds = properties.getProperty("falconInputFeeds");
-        String falconInPaths = properties.getProperty("falconInPaths");
-        String falconInputFeedStorageTypes = properties.getProperty("falconInputFeedStorageTypes");
-        String logDir = properties.getProperty("logDir");
-        String nominalTime = properties.getProperty("nominalTime");
+        String falconInputs = properties.getProperty(WorkflowExecutionArgs.INPUT_NAMES.getName());
+        String falconInPaths = properties.getProperty(WorkflowExecutionArgs.INPUT_FEED_PATHS.getName());
+        String falconInputFeedStorageTypes =
+            properties.getProperty(WorkflowExecutionArgs.INPUT_STORAGE_TYPES.getName());
+        String logDir = properties.getProperty(WorkflowExecutionArgs.LOG_DIR.getName());
+        String nominalTime = properties.getProperty(WorkflowExecutionArgs.NOMINAL_TIME.getName());
         String srcClusterName = properties.getProperty("srcClusterName");
         Path lateLogPath = handler.getLateLogPath(logDir, nominalTime, srcClusterName);
 
@@ -104,22 +106,22 @@ public class LateRerunConsumer<T extends LateRerunHandler<DelayedQueue<LaterunEv
         }
 
         String[] pathGroups = falconInPaths.split("#");
-        String[] inputFeeds = falconInputFeeds.split("#");
+        String[] inputs = falconInputs.split("#");
         String[] inputFeedStorageTypes = falconInputFeedStorageTypes.split("#");
 
         Map<String, Long> computedMetrics = new LinkedHashMap<String, Long>();
         Entity entity = EntityUtil.getEntity(message.getEntityType(), message.getEntityName());
         if (EntityUtil.getLateProcess(entity) != null) {
-            List<String> lateFeed = new ArrayList<String>();
+            List<String> lateInput = new ArrayList<String>();
             for (LateInput li : EntityUtil.getLateProcess(entity).getLateInputs()) {
-                lateFeed.add(li.getInput());
+                lateInput.add(li.getInput());
             }
 
             for (int index = 0; index < pathGroups.length; index++) {
-                if (lateFeed.contains(inputFeeds[index])) {
+                if (lateInput.contains(inputs[index])) {
                     long computedMetric = late.computeStorageMetric(
                             pathGroups[index], inputFeedStorageTypes[index], conf);
-                    computedMetrics.put(inputFeeds[index], computedMetric);
+                    computedMetrics.put(inputs[index], computedMetric);
                 }
             }
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/06f52eca/webapp/src/test/java/org/apache/falcon/late/LateDataHandlerIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/late/LateDataHandlerIT.java b/webapp/src/test/java/org/apache/falcon/late/LateDataHandlerIT.java
index ab60307..96c99c5 100644
--- a/webapp/src/test/java/org/apache/falcon/late/LateDataHandlerIT.java
+++ b/webapp/src/test/java/org/apache/falcon/late/LateDataHandlerIT.java
@@ -118,7 +118,7 @@ public class LateDataHandlerIT {
         String[] args = {
             "-out", lateDataDir,
             "-paths", feedUriTemplate,
-            "-falconInputFeeds", "foo",
+            "-falconInputNames", "foo",
             "-falconInputFeedStorageTypes", "TABLE",
         };
 
@@ -158,7 +158,7 @@ public class LateDataHandlerIT {
         String[] args = {
             "-out", lateDataDir,
             "-paths", feedUriTemplate,
-            "-falconInputFeeds", "foo",
+            "-falconInputNames", "foo",
             "-falconInputFeedStorageTypes", "TABLE",
         };
 


[04/41] git commit: FALCON-166 Instance status start and end dates are rigid and inconvenient. Contributed by Balu Vellanki

Posted by ra...@apache.org.
FALCON-166 Instance status start and end dates are rigid and inconvenient. Contributed by Balu Vellanki


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

Branch: refs/heads/FALCON-585
Commit: 1db8712d61bbdf3f24f04a4a85b18ed91e171a21
Parents: b7f5915
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Wed Aug 27 17:14:50 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Wed Aug 27 17:14:50 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   4 +
 .../java/org/apache/falcon/cli/FalconCLI.java   |  12 +-
 .../org/apache/falcon/client/FalconClient.java  |   4 +-
 .../org/apache/falcon/entity/EntityUtil.java    |  16 ++
 .../apache/falcon/entity/EntityUtilTest.java    |  17 ++
 docs/src/site/twiki/FalconCLI.twiki             |  51 ++++-
 docs/src/site/twiki/restapi/EntityList.twiki    |  37 ++--
 docs/src/site/twiki/restapi/InstanceList.twiki  |  28 +--
 docs/src/site/twiki/restapi/InstanceLogs.twiki  |  31 ++--
 .../site/twiki/restapi/InstanceRunning.twiki    |  25 +--
 .../src/site/twiki/restapi/InstanceStatus.twiki |  29 +--
 .../site/twiki/restapi/InstanceSummary.twiki    |  17 +-
 docs/src/site/twiki/restapi/ResourceList.twiki  |  12 +-
 .../falcon/resource/AbstractEntityManager.java  |  14 +-
 .../resource/AbstractInstanceManager.java       | 185 +++++++++----------
 .../resource/proxy/InstanceManagerProxy.java    |  10 +-
 .../proxy/SchedulableEntityManagerProxy.java    |   3 +-
 .../falcon/resource/EntityManagerTest.java      |  15 +-
 .../apache/falcon/resource/InstanceManager.java |   8 +-
 .../resource/SchedulableEntityManager.java      |   3 +-
 .../java/org/apache/falcon/cli/FalconCLIIT.java |  21 +++
 21 files changed, 339 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b67310b..8bd1ff9 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,7 +27,11 @@ Trunk (Unreleased)
    FALCON-263 API to get workflow parameters. (pavan kumar kolamuri via Shwetha GS)
 
   IMPROVEMENTS
+   FALCON-166 Instance status start and end dates are rigid and inconvenient
+   (Balu Vellanki via Venkatesh Seetharam)
+
    FALCON-612 Create CHANGES.txt for falcon-regression(Arpit Gupta via SamarthG)
+
    FALCON-470 Add support for pagination, filter by, etc. to Entity and
    Instance List API (Balu Vellanki via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
index b0c133a..cb46d46 100644
--- a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
+++ b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
@@ -227,8 +227,9 @@ public class FalconCLI {
         List<LifeCycle> lifeCycles = getLifeCycle(commandLine.getOptionValue(LIFECYCLE_OPT));
         String filterBy = commandLine.getOptionValue(FILTER_BY_OPT);
         String orderBy = commandLine.getOptionValue(ORDER_BY_OPT);
-        Integer offset = validateIntInput(commandLine.getOptionValue(OFFSET_OPT), 0, "offset");
-        Integer numResults = validateIntInput(commandLine.getOptionValue(NUM_RESULTS_OPT), -1, "numResults");
+        Integer offset = parseIntegerInput(commandLine.getOptionValue(OFFSET_OPT), 0, "offset");
+        Integer numResults = parseIntegerInput(commandLine.getOptionValue(NUM_RESULTS_OPT),
+                FalconClient.DEFAULT_NUM_RESULTS, "numResults");
 
         colo = getColo(colo);
         String instanceAction = "instance";
@@ -272,7 +273,7 @@ public class FalconCLI {
         OUT.get().println(result);
     }
 
-    private Integer validateIntInput(String optionValue, int defaultVal, String optionName) throws FalconCLIException {
+    private Integer parseIntegerInput(String optionValue, int defaultVal, String optionName) throws FalconCLIException {
         Integer integer = defaultVal;
         if (optionValue != null) {
             try {
@@ -343,8 +344,9 @@ public class FalconCLI {
         String filterBy = commandLine.getOptionValue(FILTER_BY_OPT);
         String filterTags = commandLine.getOptionValue(TAGS_OPT);
         String fields = commandLine.getOptionValue(FIELDS_OPT);
-        Integer offset = validateIntInput(commandLine.getOptionValue(OFFSET_OPT), 0, "offset");
-        Integer numResults =validateIntInput(commandLine.getOptionValue(NUM_RESULTS_OPT), -1, "numResults");
+        Integer offset = parseIntegerInput(commandLine.getOptionValue(OFFSET_OPT), 0, "offset");
+        Integer numResults = parseIntegerInput(commandLine.getOptionValue(NUM_RESULTS_OPT),
+                FalconClient.DEFAULT_NUM_RESULTS, "numResults");
         validateEntityType(entityType);
 
         if (optionsList.contains(SUBMIT_OPT)) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index 6697227..5e9543c 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -81,6 +81,8 @@ public class FalconClient {
     private static final String AUTH_COOKIE_EQ = AUTH_COOKIE + "=";
     private static final KerberosAuthenticator AUTHENTICATOR = new KerberosAuthenticator();
 
+    public static final int DEFAULT_NUM_RESULTS = 10;
+
     public static final HostnameVerifier ALL_TRUSTING_HOSTNAME_VERIFIER = new HostnameVerifier() {
         @Override
         public boolean verify(String hostname, SSLSession sslSession) {
@@ -571,7 +573,7 @@ public class FalconClient {
                                        String runid, String colo,
                                        List<LifeCycle> lifeCycles) throws FalconCLIException {
         return sendInstanceRequest(instances, type, entity, start, end, props,
-                runid, colo, lifeCycles, "", "", 0, -1);
+                runid, colo, lifeCycles, "", "", 0, DEFAULT_NUM_RESULTS);
     }
 
     private String sendInstanceRequest(Instances instances, String type, String entity,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
index 73e19f5..e75f28e 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -22,6 +22,7 @@ import org.apache.commons.beanutils.PropertyUtils;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.falcon.FalconException;
+import org.apache.falcon.Pair;
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.WorkflowNameBuilder.WorkflowName;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -687,4 +688,19 @@ public final class EntityUtil {
         return Storage.TYPE.TABLE == storageType;
     }
 
+    public static Pair<Date, Date> getEntityStartEndDates(Entity entityObject) {
+        Set<String> clusters = EntityUtil.getClustersDefined(entityObject);
+        Pair<Date, String> clusterMinStartDate = null;
+        Pair<Date, String> clusterMaxEndDate = null;
+        for (String cluster : clusters) {
+            if (clusterMinStartDate == null || clusterMinStartDate.first.after(getStartTime(entityObject, cluster))) {
+                clusterMinStartDate = Pair.of(getStartTime(entityObject, cluster), cluster);
+            }
+            if (clusterMaxEndDate == null || clusterMaxEndDate.first.before(getEndTime(entityObject, cluster))) {
+                clusterMaxEndDate = Pair.of(getEndTime(entityObject, cluster), cluster);
+            }
+        }
+        return new Pair<Date, Date>(clusterMinStartDate.first, clusterMaxEndDate.first);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
index d8a44ea..1613a66 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.falcon.entity;
 
+import org.apache.falcon.Pair;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.entity.v0.SchemaHelper;
@@ -212,4 +213,20 @@ public class EntityUtilTest extends AbstractTestBase {
                 frequency, tz, instance));
     }
 
+    @Test
+    public void testGetEntityStartEndDates() throws Exception {
+        Process process = (Process) EntityType.PROCESS.getUnmarshaller().unmarshal(
+                getClass().getResourceAsStream(PROCESS_XML));
+
+        Cluster cluster = new Cluster();
+        cluster.setName("testCluster");
+        cluster.setValidity(process.getClusters().getClusters().get(0).getValidity());
+
+        process.getClusters().getClusters().add(cluster);
+
+        Pair<Date, Date> startEndDates = EntityUtil.getEntityStartEndDates(process);
+        Assert.assertEquals(startEndDates.second.toString(), "Sat Dec 29 16:00:00 PST 2091");
+        Assert.assertEquals(startEndDates.first.toString(), "Tue Nov 01 17:00:00 PDT 2011");
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/FalconCLI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/FalconCLI.twiki b/docs/src/site/twiki/FalconCLI.twiki
index 2c5d9c5..77a306e 100644
--- a/docs/src/site/twiki/FalconCLI.twiki
+++ b/docs/src/site/twiki/FalconCLI.twiki
@@ -51,6 +51,11 @@ Entities of a particular type can be listed with list sub-command.
 Usage:
 $FALCON_HOME/bin/falcon entity -type [cluster|feed|process] -list
 
+Optional Args : -fields <<field1,field2>> -filterBy <<field1:value1,field2:value2>> -tags <<tagkey=tagvalue,tagkey=tagvalue>>
+-orderBy <<field>> -offset 0 -numResults 10
+
+<a href="./Restapi/EntityList.html">Optional params described here.</a>
+
 ---+++Update
 
 Update operation allows an already submitted/scheduled entity to be updated. Cluster update is currently
@@ -135,7 +140,31 @@ Example : Suppose a process has 3 instance, one has succeeded,one is in running
 {"status":"SUCCEEDED","message":"getStatus is successful","instances":[{"instance":"2012-05-07T05:02Z","status":"SUCCEEDED","logFile":"http://oozie-dashboard-url"},{"instance":"2012-05-07T05:07Z","status":"RUNNING","logFile":"http://oozie-dashboard-url"}, {"instance":"2010-01-02T11:05Z","status":"WAITING"}] 
 
 Usage:
-$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -status -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'"
+$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -status
+
+Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'" -colo <<colo>>
+-filterBy <<field1:value1,field2:value2>> -lifecycle <<lifecycles>>
+-orderBy field -offset 0 -numResults 10
+
+<a href="./Restapi/InstanceStatus.html"> Optional params described here.</a>
+
+---+++List
+
+List option via CLI can be used to get single or multiple instances.  If the instance is not yet materialized but is within the process validity range, WAITING is returned as the state. Instance time is also returned. Log location gives the oozie workflow url
+If the instance is in WAITING state, missing dependencies are listed
+
+Example : Suppose a process has 3 instance, one has succeeded,one is in running state and other one is waiting, the expected output is:
+
+{"status":"SUCCEEDED","message":"getStatus is successful","instances":[{"instance":"2012-05-07T05:02Z","status":"SUCCEEDED","logFile":"http://oozie-dashboard-url"},{"instance":"2012-05-07T05:07Z","status":"RUNNING","logFile":"http://oozie-dashboard-url"}, {"instance":"2010-01-02T11:05Z","status":"WAITING"}]
+
+Usage:
+$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -list
+
+Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'"
+-colo <<colo>> -lifecycle <<lifecycles>>
+-filterBy <<field1:value1,field2:value2>> -orderBy field -offset 0 -numResults 10
+
+<a href="./Restapi/InstanceList.html">Optional params described here.</a>
 
 ---+++Summary
 
@@ -148,7 +177,12 @@ Example : Suppose a process has 3 instance, one has succeeded,one is in running
 {"status":"SUCCEEDED","message":"getSummary is successful", "cluster": <<name>> [{"SUCCEEDED":"1"}, {"WAITING":"1"}, {"RUNNING":"1"}]}
 
 Usage:
-$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -summary -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'"
+$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -summary
+
+Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'"
+-colo <<colo>> -lifecycle <<lifecycles>>
+
+<a href="./Restapi/InstanceSummary.html">Optional params described here.</a>
 
 ---+++Running
 
@@ -157,12 +191,23 @@ Running option provides all the running instances of the mentioned process.
 Usage:
 $FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -running
 
+Optional Args : -colo <<colo>> -lifecycle <<lifecycles>>
+-filterBy <<field1:value1,field2:value2>> -orderBy <<field>> -offset 0 -numResults 10
+
+<a href="./Restapi/InstanceRunning.html">Optional params described here.</a>
+
 ---+++Logs
 
 Get logs for instance actions
 
 Usage:
-$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -logs -start "yyyy-MM-dd'T'HH:mm'Z'" [-end "yyyy-MM-dd'T'HH:mm'Z'"] [-runid <<runid>>]
+$FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -logs
+
+Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'" -runid <<runid>>
+-colo <<colo>> -lifecycle <<lifecycles>>
+-filterBy <<field1:value1,field2:value2>> -orderBy field -offset 0 -numResults 10
+
+<a href="./Restapi/InstanceLogs.html">Optional params described here.</a>
 
 ---+++LifeCycle
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/EntityList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/EntityList.twiki b/docs/src/site/twiki/restapi/EntityList.twiki
index 678b907..0697561 100644
--- a/docs/src/site/twiki/restapi/EntityList.twiki
+++ b/docs/src/site/twiki/restapi/EntityList.twiki
@@ -8,17 +8,18 @@
 Get list of the entities.
 
 ---++ Parameters
-   * :entity-type can be cluster, feed or process.
-   * :fields <optional param> additional fields that the client are interested in, separated by commas.
-      * Currently falcon supports STATUS, TAGS, PIPELINES as valid fields.
-   * :filterBy <optional param> Filter results by a given list of field,value pair. Example: filterBy=STATUS:RUNNING,PIPELINES:clickLogs
-      * Supported filter fields are TYPE, NAME, STATUS, PIPELINES
-      * Only the entities that match both filters are returned.
-   * :tags <optional param> Filter results by a given list of tags, separated by a comma. Example: tags=consumer=consumer@xyz.com,owner=producer@xyz.com
-   * :orderBy <optional param> is the field by which results should be ordered.
-      * Supports ordering by "name","type"
-   * :offset <optional param> start showing results from the offset, used for pagination. Integer, Default is 0
-   * :numResults <optional param> number of results to show per request, used for pagination. Integer, Default is -1 (means all results)
+   * :entity-type Valid options are cluster, feed or process.
+   * fields <optional param> Fields of entity that the user wants to view, separated by commas.
+      * Valid options are STATUS, TAGS, PIPELINES.
+   * filterBy <optional param> Filter results by list of field:value pairs. Example: filterBy=STATUS:RUNNING,PIPELINES:clickLogs
+      * Supported filter fields are NAME, STATUS, PIPELINES.
+      * Query will do an AND among filterBy fields.
+   * tags <optional param> Return list of entities that have specified tags, separated by a comma. Query will do AND on tag values.
+      * Example: tags=consumer=consumer@xyz.com,owner=producer@xyz.com
+   * orderBy <optional param> Field by which results should be ordered.
+      * Supports ordering by "name".
+   * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
+   * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
 
 ---++ Results
 List of the entities.
@@ -68,25 +69,25 @@ GET http://localhost:15000/api/entities/list/feed?fields=status
 
 ---+++ Rest Call
 <verbatim>
-GET http://localhost:15000/api/entities/list/feed?filterBy=STATUS:RUNNING,PIPELINES:dataReplication&fields=status,pipelines,tags&tags=consumer=consumer@xyz.com&orderBy=name&offset=2&numResults=2
+GET http://localhost:15000/api/entities/list/process?filterBy=STATUS:RUNNING,PIPELINES:dataReplication&fields=status,pipelines,tags&tags=consumer=consumer@xyz.com&orderBy=name&offset=2&numResults=2
 </verbatim>
 ---+++ Result
 <verbatim>
 {
     "entity": [
         {
-            "name"  : "SampleInput2",
-            "type"  : "feed",
+            "name"  : "SampleProcess1",
+            "type"  : "process",
             "status": "RUNNING",
             "pipelines": "dataReplication",
-            "tags":consumer=consumer@xyz.com
+            "tags": "consumer=consumer@xyz.com"
         },
         {
-            "name": "SampleInput3",
-            "type": "feed",
+            "name": "SampleProcess3",
+            "type": "process",
             "status": "RUNNING",
             "pipelines": "dataReplication",
-            "tags":consumer=consumer@xyz.com,owner=producer@xyz.com
+            "tags": "consumer=consumer@xyz.com,owner=producer@xyz.com"
         }
     ]
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/InstanceList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceList.twiki b/docs/src/site/twiki/restapi/InstanceList.twiki
index c2059f2..76eaa33 100644
--- a/docs/src/site/twiki/restapi/InstanceList.twiki
+++ b/docs/src/site/twiki/restapi/InstanceList.twiki
@@ -8,19 +8,21 @@
 Get list of all instances of a given entity.
 
 ---++ Parameters
-   * :entity-type can either be a feed or a process.
-   * :entity-name is name of the entity.
-   * start is the start time of the instance that you want to refer to
-   * end <optional param> is the end time of the instance that you want to refer to
-   * colo <optional param> colo on which the query should be run
-   * lifecycle <optional param> can be Eviction/Replication(default) for feed and Execution(default) for process.
-   * filterBy <optional param> Filter results by a given list of field,value pairs. Example: filterBy=STATUS:RUNNING,CLUSTER:primary-cluster
-      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER,STARTEDAFTER
-      * Only the instances that match both filters are returned.
-   * orderBy <optional param> is the instance field by which results should be ordered.
-      * Supports ordering by "status","startTime","endTime","cluster"
-   * offset <optional param> start showing results from the offset, used for pagination. Integer, Default is 0
-   * numResults <optional param> number of results to show per request, used for pagination. Integer, Default is -1 (means all results)
+   * :entity-type Valid options are cluster, feed or process.
+   * :entity-name Name of the entity.
+   * start <optional param> Show instances from this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * By default, it is set to (end - (10 * entityFrequency)).
+   * end <optional param> Show instances up to this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * Default is set to now.
+   * colo <optional param> Colo on which the query should be run.
+   * lifecycle <optional param> Valid lifecycles for feed are Eviction/Replication(default) and for process is Execution(default).
+   * filterBy <optional param>  Filter results by list of field:value pairs. Example: filterBy=STATUS:RUNNING,CLUSTER:primary-cluster
+      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER, STARTEDAFTER.
+      * Query will do an AND among filterBy fields.
+   * orderBy <optional param> Field by which results should be ordered.
+      * Supports ordering by  "status","startTime","endTime","cluster".
+   * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
+   * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
    
 ---++ Results
 List of instances of given entity.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/InstanceLogs.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceLogs.twiki b/docs/src/site/twiki/restapi/InstanceLogs.twiki
index 56d5fe1..354eed4 100644
--- a/docs/src/site/twiki/restapi/InstanceLogs.twiki
+++ b/docs/src/site/twiki/restapi/InstanceLogs.twiki
@@ -8,20 +8,23 @@
 Get log of a specific instance of an entity.
 
 ---++ Parameters
-   * :entity-type can either be a feed or a process.
-   * :entity-name is name of the entity.
-   * start is the start time of the instance that you want to refer to
-   * end <optional param> is the end time of the instance that you want to refer to
-   * colo <optional param> colo on which the query should be run
-   * lifecycle <optional param> can be Eviction/Replication(default) for feed and Execution(default) for process.
-   * filterBy <optional param> Filter results by a given list of field,value pairs. Example: filterBy=STATUS:SUCCEEDED,CLUSTER:primary-cluster
-      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER,STARTEDAFTER
-      * Only the instances that match both filters are returned.
-   * orderBy <optional param> is the instance field by which results should be ordered.
-      * Supports ordering by "status","startTime","endTime","cluster"
-   * offset <optional param> start showing results from the offset, used for pagination. Integer, Default is 0
-   * numResults <optional param> number of results to show per request, used for pagination. Integer, Default is -1 (means all results)
-  
+   * :entity-type Valid options are cluster, feed or process.
+   * :entity-name Name of the entity.
+   * start <optional param> Show instances from this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * By default, it is set to (end - (10 * entityFrequency)).
+   * end <optional param> Show instances up to this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * Default is set to now.
+   * colo <optional param> Colo on which the query should be run.
+   * runId <optional param> Run Id.
+   * lifecycle <optional param> Valid lifecycles for feed are Eviction/Replication(default) and for process is Execution(default).
+   * filterBy <optional param>  Filter results by list of field:value pairs. Example: filterBy=STATUS:RUNNING,CLUSTER:primary-cluster
+      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER, STARTEDAFTER.
+      * Query will do an AND among filterBy fields.
+   * orderBy <optional param> Field by which results should be ordered.
+      * Supports ordering by "status","startTime","endTime","cluster".
+   * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
+   * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
+
 ---++ Results
 Log of specified instance.
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/InstanceRunning.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceRunning.twiki b/docs/src/site/twiki/restapi/InstanceRunning.twiki
index 0d04008..47037d6 100644
--- a/docs/src/site/twiki/restapi/InstanceRunning.twiki
+++ b/docs/src/site/twiki/restapi/InstanceRunning.twiki
@@ -8,17 +8,18 @@
 Get a list of instances currently running for a given entity.
 
 ---++ Parameters
-   * :entity-type can either be a feed or a process.
-   * :entity-name is name of the entity.
-   * colo <optional param> colo on which the query should be run
-   * lifecycle <optional param> can be Eviction/Replication(default) for feed and Execution(default) for process.
-   * filterBy <optional param> Filter results by a given list of field,value pairs. Example: filterBy=STATUS:RUNNING,CLUSTER:primary-cluster
-      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER,STARTEDAFTER
-      * Only the instances that match both filters are returned.
-   * orderBy <optional param> is the instance field by which results should be ordered.
-      * Supports ordering by "status","startTime","endTime","cluster"
-   * offset <optional param> start showing results from the offset, used for pagination. Integer, Default is 0
-   * numResults <optional param> number of results to show per request, used for pagination. Integer, Default is -1 (means all results)
+   * :entity-type Valid options are cluster, feed or process.
+   * :entity-name Name of the entity.
+   * colo <optional param> Colo on which the query should be run.
+   * lifecycle <optional param> Valid lifecycles for feed are Eviction/Replication(default) and for process is Execution(default).
+   * filterBy <optional param>  Filter results by list of field:value pairs. Example: filterBy=CLUSTER:primary-cluster
+      * Supported filter fields are CLUSTER, SOURCECLUSTER, STARTEDAFTER.
+      * Query will do an AND among filterBy fields.
+   * orderBy <optional param> Field by which results should be ordered.
+      * Supports ordering by "status","startTime","endTime","cluster".
+   * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
+   * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
+
 
 ---++ Results
 List of instances currently running.
@@ -48,7 +49,7 @@ GET http://localhost:15000/api/instance/running/process/SampleProcess?colo=*
 
 ---+++ Rest Call
 <verbatim>
-GET http://localhost:15000/api/instance/running/process/SampleProcess?colo=*&start=2012-04-03T07:00Z&filterBy=STATUS:RUNNING,CLUSTER:primary-cluster&orderBy=startTime&offset=2&numResults=2
+GET http://localhost:15000/api/instance/running/process/SampleProcess?colo=*&start=2012-04-03T07:00Z&filterBy=CLUSTER:primary-cluster&orderBy=startTime&offset=2&numResults=2
 </verbatim>
 ---+++ Result
 <verbatim>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/InstanceStatus.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceStatus.twiki b/docs/src/site/twiki/restapi/InstanceStatus.twiki
index 5160252..69634de 100644
--- a/docs/src/site/twiki/restapi/InstanceStatus.twiki
+++ b/docs/src/site/twiki/restapi/InstanceStatus.twiki
@@ -8,19 +8,22 @@
 Get status of a specific instance of an entity.
 
 ---++ Parameters
-   * :entity-type can either be a feed or a process.
-   * :entity-name is name of the entity.
-   * start is the start time of the instance that you want to refer to
-   * end <optional param> is the end time of the instance that you want to refer to
-   * colo <optional param> colo on which the query should be run
-   * lifecycle <optional param> can be Eviction/Replication(default) for feed and Execution(default) for process.
-   * filterBy <optional param> Filter results by a given list of field,value pairs. Example: filterBy=STATUS:RUNNING,CLUSTER:primary-cluster
-      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER,STARTEDAFTER
-      * Only the instances that match both filters are returned.
-   * orderBy <optional param> is the instance field by which results should be ordered.
-      * Supports ordering by "status","startTime","endTime","cluster"
-   * offset <optional param> start showing results from the offset, used for pagination. Integer, Default is 0
-   * numResults <optional param> number of results to show per request, used for pagination. Integer, Default is -1 (means all results)
+   * :entity-type Valid options are cluster, feed or process.
+   * :entity-name Name of the entity.
+   * start <optional param> Show instances from this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * By default, it is set to (end - (10 * entityFrequency)).
+   * end <optional param> Show instances up to this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * Default is set to now.
+   * colo <optional param> Colo on which the query should be run.
+   * lifecycle <optional param> Valid lifecycles for feed are Eviction/Replication(default) and for process is Execution(default).
+   * filterBy <optional param>  Filter results by list of field:value pairs. Example: filterBy=STATUS:RUNNING,CLUSTER:primary-cluster
+      * Supported filter fields are STATUS, CLUSTER, SOURCECLUSTER, STARTEDAFTER.
+      * Query will do an AND among filterBy fields.
+   * orderBy <optional param> Field by which results should be ordered.
+      * Supports ordering by "status","startTime","endTime","cluster".
+   * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
+   * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
+
    
 ---++ Results
 Status of the specified instance.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/InstanceSummary.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceSummary.twiki b/docs/src/site/twiki/restapi/InstanceSummary.twiki
index 4fa5780..9d1041e 100644
--- a/docs/src/site/twiki/restapi/InstanceSummary.twiki
+++ b/docs/src/site/twiki/restapi/InstanceSummary.twiki
@@ -8,12 +8,14 @@
 Get summary of instance/instances of an entity.
 
 ---++ Parameters
-   * :entity-type can either be a feed or a process.
-   * :entity-name is name of the entity.
-   * start is the start time of the instance that you want to refer to
-   * end <optional param> is the end time of the instance that you want to refer to
-   * colo <optional param> colo on which the query should be run
-   * lifecycle <optional param> can be Eviction/Replication(default) for feed and Execution(default) for process.
+   * :entity-type Valid options are cluster, feed or process.
+   * :entity-name Name of the entity.
+   * start <optional param> Show instances from this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * By default, it is set to (end - (10 * entityFrequency)).
+   * end <optional param> Show instances up to this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * Default is set to now.
+   * colo <optional param> Colo on which the query should be run.
+   * lifecycle <optional param> Valid lifecycles for feed are Eviction/Replication(default) and for process is Execution(default).
 
 ---++ Results
 Summary of the instances over the specified time range
@@ -22,7 +24,6 @@ Summary of the instances over the specified time range
 ---+++ Rest Call
 <verbatim>
 GET http://localhost:15000/api/instance/summary/process/WordCount?colo=*&start=2014-01-21T13:00Z&end=2014-01-21T16:00Z
-Remote-User: suhas
 </verbatim>
 ---+++ Result
 <verbatim>
@@ -38,7 +39,7 @@ Remote-User: suhas
                     "entry":
                         {
                             "key":"SUCCEEDED",
-                            "value":"3"
+                            "key2":"value"
                          }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/docs/src/site/twiki/restapi/ResourceList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/ResourceList.twiki b/docs/src/site/twiki/restapi/ResourceList.twiki
index 4b377e8..9284810 100644
--- a/docs/src/site/twiki/restapi/ResourceList.twiki
+++ b/docs/src/site/twiki/restapi/ResourceList.twiki
@@ -48,21 +48,21 @@ See also: [[../Security.twiki][Security in Falcon]]
 | DELETE      | [[EntityDelete][api/entities/delete/:entity-type/:entity-name]]             | Delete the entity                  |
 | GET         | [[EntityStatus][api/entities/status/:entity-type/:entity-name]]             | Get the status of the entity       |
 | GET         | [[EntityDefinition][api/entities/definition/:entity-type/:entity-name]]     | Get the definition of the entity   |
-| GET         | [[EntityList][api/entities/list/:entity-type?fields=:fields&filterBy=:filterBy&tags=:tags&orderBy=:orderBy&offset=:offset&numResults=:numResults]]               | Get the list of entities           |
+| GET         | [[EntityList][api/entities/list/:entity-type]]                              | Get the list of entities           |
 | GET         | [[EntityDependencies][api/entities/dependencies/:entity-type/:entity-name]] | Get the dependencies of the entity |
 
 ---++ REST Call on Feed and Process Instances
 
 | *Call Type* | *Resource*                                                                  | *Description*                |
-| GET         | [[InstanceRunning][api/instance/running/:entity-type/:entity-name?colo=:colo&lifecycle=:lifecycle&...]] | List of running instances.   |
-| GET         | [[InstanceList][api/instance/list/:entity-type/:entity-name?start=:start&end=:end&colo&lifecycle=:lifecycle&...]]   | List of instances   |
-| GET         | [[InstanceStatus][api/instance/status/:entity-type/:entity-name?start=:start&end=:end&colo=:colo&lifecycle=:lifecycle&...]]   | Status of a given instance   |
+| GET         | [[InstanceRunning][api/instance/running/:entity-type/:entity-name]]         | List of running instances.   |
+| GET         | [[InstanceList][api/instance/list/:entity-type/:entity-name]]               | List of instances   |
+| GET         | [[InstanceStatus][api/instance/status/:entity-type/:entity-name]]           | Status of a given instance   |
 | POST        | [[InstanceKill][api/instance/kill/:entity-type/:entity-name]]               | Kill a given instance        |
 | POST        | [[InstanceSuspend][api/instance/suspend/:entity-type/:entity-name]]         | Suspend a running instance   |
 | POST        | [[InstanceResume][api/instance/resume/:entity-type/:entity-name]]           | Resume a given instance      |
 | POST        | [[InstanceRerun][api/instance/rerun/:entity-type/:entity-name]]             | Rerun a given instance       |
-| GET         | [[InstanceLogs][api/instance/logs/:entity-type/:entity-name?start=:start&end=:end&colo=:colo&lifecycle=:lifecycle&...]]       | Get logs of a given instance |
-| GET         | [[InstanceSummary][api/instance/summary/:entity-type/:entity-name?start=:start&end=:end&colo=:colo&lifecycle=:lifecycle]] | Return summary of instances for an entity |
+| GET         | [[InstanceLogs][api/instance/logs/:entity-type/:entity-name]]               | Get logs of a given instance |
+| GET         | [[InstanceSummary][api/instance/summary/:entity-type/:entity-name]]         | Return summary of instances for an entity |
 
 ---++ REST Call on Lineage Graph
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
index 84ae446..a6d1b29 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
@@ -60,7 +60,9 @@ import java.util.*;
 public abstract class AbstractEntityManager {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractEntityManager.class);
     private static final Logger AUDIT = LoggerFactory.getLogger("AUDIT");
+
     protected static final int XML_DEBUG_LEN = 10 * 1024;
+    protected static final String DEFAULT_NUM_RESULTS = "10";
 
     private AbstractWorkflowEngine workflowEngine;
     protected ConfigurationStore configStore = ConfigurationStore.get();
@@ -728,14 +730,22 @@ public abstract class AbstractEntityManager {
               elements[offset] is included. Size 10, offset 10, return empty list.
               Size 10, offset 5, count 3, return elements[5,6,7].
               Size 10, offset 5, count >= 5, return elements[5,6,7,8,9]
-              When count is -1, return elements starting from elements[offset] until the end */
+              return elements starting from elements[offset] until the end OR offset+numResults*/
+
+        if (numresults < 1) {
+            LOG.error("Value for param numResults should be > than 0  : {}", numresults);
+            throw FalconWebException.newException("Value for param numResults should be > than 0  : " + numresults,
+                    Response.Status.BAD_REQUEST);
+        }
+
+        if (offset < 0) { offset = 0; }
 
         if (offset >= arraySize || arraySize == 0) {
             // No elements to return
             return 0;
         }
         int retLen = arraySize - offset;
-        if (retLen > numresults && numresults != -1) {
+        if (retLen > numresults) {
             retLen = numresults;
         }
         return retLen;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
index 141ee9c..6862b99 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
@@ -24,6 +24,7 @@ import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.parser.ValidationException;
 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.entity.v0.SchemaHelper;
 import org.apache.falcon.logging.LogProvider;
 import org.apache.falcon.resource.InstancesResult.Instance;
@@ -43,6 +44,11 @@ import java.util.*;
 public abstract class AbstractInstanceManager extends AbstractEntityManager {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractInstanceManager.class);
 
+    private static final long MINUTE_IN_MILLIS = 60000L;
+    private static final long HOUR_IN_MILLIS = 3600000L;
+    private static final long DAY_IN_MILLIS = 86400000L;
+    private static final long MONTH_IN_MILLIS = 2592000000L;
+
     protected void checkType(String type) {
         if (StringUtils.isEmpty(type)) {
             throw FalconWebException.newInstanceException("entity type is empty",
@@ -111,13 +117,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         checkType(type);
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
-            validateParams(type, entity, startStr, endStr);
-            Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startStr, endStr);
+
             // LifeCycle lifeCycleObject = EntityUtil.getLifeCycle(lifeCycle);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return getInstanceResultSubset(wfEngine.getStatus(entityObject, start, end, lifeCycles),
+            return getInstanceResultSubset(wfEngine.getStatus(entityObject,
+                            startAndEndDate.first, startAndEndDate.second, lifeCycles),
                     filterBy, orderBy, offset, numResults);
         } catch (Throwable e) {
             LOG.error("Failed to get instances status", e);
@@ -132,14 +139,12 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         checkType(type);
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
-            validateParams(type, entity, startStr, endStr);
-
-            Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startStr, endStr);
 
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return wfEngine.getSummary(entityObject, start, end, lifeCycles);
+            return wfEngine.getSummary(entityObject, startAndEndDate.first, startAndEndDate.second, lifeCycles);
         } catch (Throwable e) {
             LOG.error("Failed to get instances status", e);
             throw FalconWebException.newInstanceSummaryException(e, Response.Status.BAD_REQUEST);
@@ -276,14 +281,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
             Collections.sort(instanceSet, new Comparator<Instance>() {
                 @Override
                 public int compare(Instance i1, Instance i2) {
-                    return i1.getStartTime().compareTo(i2.getStartTime());
+                    return i2.getStartTime().compareTo(i1.getStartTime()); //default desc
                 }
             });
         } else if (orderBy.equals("endTime")) {
             Collections.sort(instanceSet, new Comparator<Instance>() {
                 @Override
                 public int compare(Instance i1, Instance i2) {
-                    return i1.getEndTime().compareTo(i2.getEndTime());
+                    return i2.getEndTime().compareTo(i1.getEndTime()); //default desc
                 }
             });
         }//Default : no sort
@@ -303,13 +308,12 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
                 throw new FalconException("For displaying wf-params there can't be more than one lifecycle "
                         + lifeCycles);
             }
-            validateParams(type, entity, startTime, null);
-
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
-            Date start = EntityUtil.parseDateUTC(startTime);
-            Date end = getEndDate(start, null);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startTime, null);
+
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return wfEngine.getInstanceParams(entityObject, start, end, lifeCycles);
+            return wfEngine.getInstanceParams(entityObject, startAndEndDate.first, startAndEndDate.second, lifeCycles);
         } catch (Throwable e) {
             LOG.error("Failed to display params of an instance", e);
             throw FalconWebException.newInstanceException(e, Response.Status.BAD_REQUEST);
@@ -326,15 +330,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
             audit(request, entity, type, "INSTANCE_KILL");
-            validateParams(type, entity, startStr, endStr);
-
-            Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startStr, endStr);
 
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return wfEngine.killInstances(entityObject, start, end, props, lifeCycles);
+            return wfEngine.killInstances(entityObject,
+                    startAndEndDate.first, startAndEndDate.second, props, lifeCycles);
         } catch (Throwable e) {
             LOG.error("Failed to kill instances", e);
             throw FalconWebException.newInstanceException(e, Response.Status.BAD_REQUEST);
@@ -351,15 +354,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
             audit(request, entity, type, "INSTANCE_SUSPEND");
-            validateParams(type, entity, startStr, endStr);
-
-            Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startStr, endStr);
 
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return wfEngine.suspendInstances(entityObject, start, end, props, lifeCycles);
+            return wfEngine.suspendInstances(entityObject,
+                    startAndEndDate.first, startAndEndDate.second, props, lifeCycles);
         } catch (Throwable e) {
             LOG.error("Failed to suspend instances", e);
             throw FalconWebException.newInstanceException(e, Response.Status.BAD_REQUEST);
@@ -376,15 +378,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
             audit(request, entity, type, "INSTANCE_RESUME");
-            validateParams(type, entity, startStr, endStr);
-
-            Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startStr, endStr);
 
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return wfEngine.resumeInstances(entityObject, start, end, props, lifeCycles);
+            return wfEngine.resumeInstances(entityObject,
+                    startAndEndDate.first, startAndEndDate.second, props, lifeCycles);
         } catch (Throwable e) {
             LOG.error("Failed to resume instances", e);
             throw FalconWebException.newInstanceException(e, Response.Status.BAD_REQUEST);
@@ -400,15 +401,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
             audit(request, entity, type, "INSTANCE_RERUN");
-            validateParams(type, entity, startStr, endStr);
-
-            Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);
+            validateParams(type, entity);
             Entity entityObject = EntityUtil.getEntity(type, entity);
+            Pair<Date, Date> startAndEndDate = getStartAndEndDate(entityObject, startStr, endStr);
 
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-            return wfEngine.reRunInstances(entityObject, start, end, props, lifeCycles);
+            return wfEngine.reRunInstances(entityObject,
+                    startAndEndDate.first, startAndEndDate.second, props, lifeCycles);
         } catch (Exception e) {
             LOG.error("Failed to rerun instances", e);
             throw FalconWebException.newInstanceException(e, Response.Status.BAD_REQUEST);
@@ -427,75 +427,72 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         return props;
     }
 
-    private Date getEndDate(Date start, String endStr) throws FalconException {
-        Date end;
-        if (StringUtils.isEmpty(endStr)) {
-            end = new Date(start.getTime() + 1000); // next sec
-        } else {
-            end = EntityUtil.parseDateUTC(endStr);
+    private Pair<Date, Date> getStartAndEndDate(Entity entityObject, String startStr, String endStr)
+        throws FalconException {
+        Pair<Date, Date> clusterStartEndDates = EntityUtil.getEntityStartEndDates(entityObject);
+        Frequency frequency = EntityUtil.getFrequency(entityObject);
+        Date endDate = getEndDate(endStr, clusterStartEndDates.second);
+        Date startDate = getStartDate(startStr, endDate, clusterStartEndDates.first, frequency);
+
+        if (startDate.after(endDate)) {
+            throw new FalconException("Specified End date " + SchemaHelper.getDateFormat().format(endDate)
+                    + " is before the entity was scheduled " + SchemaHelper.getDateFormat().format(startDate));
         }
-        return end;
+        return new Pair<Date, Date>(startDate, endDate);
     }
 
-    private void validateParams(String type, String entity, String startStr,
-                                String endStr) throws FalconException {
-        validateNotEmpty("entityType", type);
-        validateNotEmpty("entityName", entity);
-        validateNotEmpty("start", startStr);
-
-        Entity entityObject = EntityUtil.getEntity(type, entity);
-        validateDateRange(entityObject, startStr, endStr);
+    private Date getEndDate(String endStr, Date clusterEndDate) throws FalconException {
+        Date endDate = StringUtils.isEmpty(endStr) ? new Date() : EntityUtil.parseDateUTC(endStr);
+        if (endDate.after(clusterEndDate)) {
+            endDate = clusterEndDate;
+        }
+        return endDate;
     }
 
-    private void validateDateRange(Entity entity, String start, String end) throws FalconException {
-        Set<String> clusters = EntityUtil.getClustersDefined(entity);
-        Pair<Date, String> clusterMinStartDate = null;
-        Pair<Date, String> clusterMaxEndDate = null;
-        for (String cluster : clusters) {
-            if (clusterMinStartDate == null || clusterMinStartDate.first.after(
-                    EntityUtil.getStartTime(entity, cluster))) {
-                clusterMinStartDate = Pair.of(EntityUtil.getStartTime(entity, cluster), cluster);
-            }
-            if (clusterMaxEndDate == null || clusterMaxEndDate.first.before(EntityUtil.getEndTime(entity, cluster))) {
-                clusterMaxEndDate = Pair.of(EntityUtil.getEndTime(entity, cluster), cluster);
+    private Date getStartDate(String startStr, Date end,
+                              Date clusterStartDate, Frequency frequency) throws FalconException {
+        Date start;
+        final int dateMultiplier = 10;
+        if (StringUtils.isEmpty(startStr)) {
+            // set startDate to endDate - 10 times frequency
+            long startMillis = end.getTime();
+
+            switch (frequency.getTimeUnit().getCalendarUnit()){
+            case Calendar.MINUTE :
+                startMillis -= frequency.getFrequencyAsInt() * MINUTE_IN_MILLIS * dateMultiplier;
+                break;
+
+            case Calendar.HOUR :
+                startMillis -= frequency.getFrequencyAsInt() * HOUR_IN_MILLIS * dateMultiplier;
+                break;
+
+            case Calendar.DATE :
+                startMillis -= frequency.getFrequencyAsInt() * DAY_IN_MILLIS * dateMultiplier;
+                break;
+
+            case Calendar.MONTH :
+                startMillis -= frequency.getFrequencyAsInt() * MONTH_IN_MILLIS * dateMultiplier;
+                break;
+
+            default:
+                break;
             }
-        }
 
-        validateDateRangeFor(entity, clusterMinStartDate, clusterMaxEndDate,
-                start, end);
-    }
-
-    private void validateDateRangeFor(Entity entity, Pair<Date, String> clusterMinStart,
-                                      Pair<Date, String> clusterMaxEnd, String start,
-                                      String end) throws FalconException{
+            start = new Date(startMillis);
+        } else {
+            start = EntityUtil.parseDateUTC(startStr);
+        }
 
-        Date instStart = EntityUtil.parseDateUTC(start);
-        if (instStart.before(clusterMinStart.first)) {
-            throw new ValidationException("Start date " + start + " is before "
-                    + entity.getEntityType() + "'s  start "
-                    + SchemaHelper.formatDateUTC(clusterMinStart.first)
-                    + " for cluster " + clusterMinStart.second);
+        if (start.before(clusterStartDate)) {
+            start = clusterStartDate;
         }
 
-        if (StringUtils.isNotEmpty(end)) {
-            Date instEnd = EntityUtil.parseDateUTC(end);
-            if (instStart.after(instEnd)) {
-                throw new ValidationException("Start date " + start
-                        + " is after end date " + end);
-            }
+        return start;
+    }
 
-            if (instEnd.after(clusterMaxEnd.first)) {
-                throw new ValidationException("End date " + end + " is after "
-                        + entity.getEntityType() + "'s end "
-                        + SchemaHelper.formatDateUTC(clusterMaxEnd.first)
-                        + " for cluster " + clusterMaxEnd.second);
-            }
-        } else if (instStart.after(clusterMaxEnd.first)) {
-            throw new ValidationException("Start date " + start + " is after "
-                    + entity.getEntityType() + "'s end "
-                    + SchemaHelper.formatDateUTC(clusterMaxEnd.first)
-                    + " for cluster " + clusterMaxEnd.second);
-        }
+    private void validateParams(String type, String entity) throws FalconException {
+        validateNotEmpty("entityType", type);
+        validateNotEmpty("entityName", entity);
     }
 
     private void validateNotEmpty(String field, String param) throws ValidationException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
index 970ea74..bfce5f9 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
@@ -45,7 +45,7 @@ import java.util.*;
  */
 @Path("instance")
 public class InstanceManagerProxy extends AbstractInstanceManager {
-
+    private static final String DEFAULT_NUM_RESULTS = "10";
     private final Map<String, Channel> processInstanceManagerChannels = new HashMap<String, Channel>();
 
     public InstanceManagerProxy() {
@@ -85,7 +85,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") final Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
@@ -115,7 +115,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") final Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
@@ -141,7 +141,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") final Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
@@ -210,7 +210,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") final Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
index 7b8bdf5..f743005 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
@@ -301,7 +301,8 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
                                     @DefaultValue("") @QueryParam("tags") String tags,
                                     @DefaultValue("") @QueryParam("orderBy") String orderBy,
                                     @DefaultValue("0") @QueryParam("offset") Integer offset,
-                                    @DefaultValue("-1") @QueryParam("numResults") Integer resultsPerPage) {
+                                    @DefaultValue(DEFAULT_NUM_RESULTS)
+                                    @QueryParam("numResults") Integer resultsPerPage) {
         return super.getEntityList(type, fields, filterBy, tags, orderBy, offset, resultsPerPage);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
index e8c54f3..2c0fa25 100644
--- a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
+++ b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
@@ -114,7 +114,7 @@ public class EntityManagerTest extends AbstractEntityManager {
         Entity process2 = buildProcess("processAuthUser", System.getProperty("user.name"), "", "");
         configStore.publish(EntityType.PROCESS, process2);
 
-        EntityList entityList = this.getEntityList("process", "", "", "", "", 0, -1);
+        EntityList entityList = this.getEntityList("process", "", "", "", "", 0, 10);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 2);
 
@@ -123,7 +123,7 @@ public class EntityManagerTest extends AbstractEntityManager {
          */
         StartupProperties.get().setProperty("falcon.security.authorization.enabled", "true");
         CurrentUser.authenticate(System.getProperty("user.name"));
-        entityList = this.getEntityList("process", "", "", "", "", 0, -1);
+        entityList = this.getEntityList("process", "", "", "", "", 0, 10);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 2);
 
@@ -131,7 +131,7 @@ public class EntityManagerTest extends AbstractEntityManager {
          * Only one entity should be returned when the auth is enabled.
          */
         CurrentUser.authenticate("fakeUser");
-        entityList = this.getEntityList("process", "", "", "", "", 0, -1);
+        entityList = this.getEntityList("process", "", "", "", "", 0, 10);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 1);
 
@@ -182,6 +182,15 @@ public class EntityManagerTest extends AbstractEntityManager {
         entityList = this.getEntityList("process", "pipelines", "",
                 "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", 10, 2);
         Assert.assertEquals(entityList.getElements().length, 0);
+
+        // Test negative value for numResults, should throw an exception.
+        try {
+            this.getEntityList("process", "pipelines", "",
+                    "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", 10, -1);
+            Assert.assertTrue(false);
+        } catch (Throwable e) {
+            Assert.assertTrue(true);
+        }
     }
 
     private Entity buildProcess(String name, String username, String tags, String pipelines) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
index 8762cc1..a2aaac2 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
@@ -48,7 +48,7 @@ public class InstanceManager extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getRunningInstances(type, entity, colo, lifeCycles, filterBy, orderBy, offset, resultsPerPage);
     }
 
@@ -71,7 +71,7 @@ public class InstanceManager extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getInstances(type, entity, startStr, endStr, colo, lifeCycles,
                 filterBy, orderBy, offset, resultsPerPage);
     }
@@ -91,7 +91,7 @@ public class InstanceManager extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getStatus(type, entity, startStr, endStr, colo, lifeCycles,
                 filterBy, orderBy, offset, resultsPerPage);
     }
@@ -126,7 +126,7 @@ public class InstanceManager extends AbstractInstanceManager {
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
-            @DefaultValue("-1") @QueryParam("numResults") Integer resultsPerPage) {
+            @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getLogs(type, entity, startStr, endStr, colo, runId, lifeCycles,
                 filterBy, orderBy, offset, resultsPerPage);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
index f993f81..58a3bd2 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
@@ -64,7 +64,8 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
                                     @DefaultValue("") @QueryParam("tags") String tags,
                                     @DefaultValue("") @QueryParam("orderBy") String orderBy,
                                     @DefaultValue("0") @QueryParam("offset") Integer offset,
-                                    @DefaultValue("-1") @QueryParam("numResults") Integer resultsPerPage) {
+                                    @DefaultValue(DEFAULT_NUM_RESULTS)
+                                    @QueryParam("numResults") Integer resultsPerPage) {
         return super.getEntityList(type, fields, filterBy, tags, orderBy, offset, resultsPerPage);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1db8712d/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
index b65aa46..4ac978c 100644
--- a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
+++ b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
@@ -405,6 +405,27 @@ public class FalconCLIIT {
                         + overlay.get("outputFeedName")
                         + " -start " + SchemaHelper.getDateFormat().format(new Date())
                         + " -filterBy INVALID:FILTER -offset 0 -numResults 1"));
+
+        // testcase : start str is older than entity schedule time.
+        Assert.assertEquals(0,
+                executeWithURL("instance -running -type feed -lifecycle eviction -name "
+                        + overlay.get("outputFeedName")
+                        + " -start " + SchemaHelper.getDateFormat().format(new Date(10000))
+                        + " -orderBy startTime -offset 0 -numResults 1"));
+        // testcase : end str is in future
+        long futureTimeinMilliSecs = (new Date()).getTime()+ 86400000;
+        Assert.assertEquals(0,
+                executeWithURL("instance -running -type feed -lifecycle eviction -name "
+                        + overlay.get("outputFeedName")
+                        + " -start " + SchemaHelper.getDateFormat().format(new Date(10000))
+                        + " -end " + SchemaHelper.getDateFormat().format(new Date(futureTimeinMilliSecs))
+                        + " -orderBy startTime -offset 0 -numResults 1"));
+        // Both start and end dates are optional
+        Assert.assertEquals(0,
+                executeWithURL("instance -running -type feed -lifecycle eviction -name "
+                        + overlay.get("outputFeedName")
+                        + " -orderBy startTime -offset 0 -numResults 1"));
+
         Assert.assertEquals(0,
                 executeWithURL("instance -status -type process -name "
                         + overlay.get("processName")


[23/41] git commit: FALCON-654 Exclude junit dependency in pom

Posted by ra...@apache.org.
FALCON-654 Exclude junit dependency in pom


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

Branch: refs/heads/FALCON-585
Commit: 9f4a6d2dfece9ed5da21b085a8d6b4f1f4ad18b7
Parents: 3678eab
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Mon Sep 1 13:18:51 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Wed Sep 3 18:25:58 2014 +0300

----------------------------------------------------------------------
 CHANGES.txt               | 2 ++
 falcon-regression/pom.xml | 4 ----
 pom.xml                   | 6 ++++++
 3 files changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9f4a6d2d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f554236..db2dd04 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,8 @@ Trunk (Unreleased)
    FALCON-263 API to get workflow parameters. (pavan kumar kolamuri via Shwetha GS)
 
   IMPROVEMENTS
+   FALCON-654 Exclude junit dependency in pom (Ruslan Ostafiychuk)
+
    FALCON-640 Add ability to specify sort order for orderBy param in RestAPI
    (Balu Vellanki via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9f4a6d2d/falcon-regression/pom.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/pom.xml b/falcon-regression/pom.xml
index e5bb31a..b1931d8 100644
--- a/falcon-regression/pom.xml
+++ b/falcon-regression/pom.xml
@@ -178,10 +178,6 @@
                         <artifactId>slf4j-simple</artifactId>
                     </exclusion>
                     <exclusion>
-                        <groupId>com.googlecode.json-simple</groupId>
-                        <artifactId>json-simple</artifactId>
-                    </exclusion>
-                    <exclusion>
                         <groupId>org.codehaus.jackson</groupId>
                         <artifactId>jackson-core-asl</artifactId>
                     </exclusion>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9f4a6d2d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5b2bc4f..28c09a0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -696,6 +696,12 @@
                 <groupId>com.googlecode.json-simple</groupId>
                 <artifactId>json-simple</artifactId>
                 <version>1.1.1</version>
+                <exclusions>
+                    <exclusion>
+                        <groupId>junit</groupId>
+                        <artifactId>junit</artifactId>
+                    </exclusion>
+                </exclusions>
             </dependency>
 
             <dependency>


[06/41] git commit: FALCON-632 Refactoring, documentation stuff contributed by Paul Isaychuk

Posted by ra...@apache.org.
FALCON-632 Refactoring, documentation stuff contributed by Paul Isaychuk


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

Branch: refs/heads/FALCON-585
Commit: 5dfe5cdef7521d2e41de32d8abd35f9a389ae82a
Parents: 78b9c1a
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Thu Aug 28 11:33:05 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Thu Aug 28 11:33:05 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   2 +
 .../regression/EmbeddedPigScriptTest.java       |  60 +++----
 .../regression/FeedInstanceStatusTest.java      | 170 ++++++++-----------
 .../regression/ProcessInstanceKillsTest.java    |  62 +++----
 .../regression/ProcessInstanceRerunTest.java    | 127 ++++++--------
 .../falcon/regression/ProcessLibPathTest.java   |  40 ++---
 .../falcon/regression/prism/RetentionTest.java  |  67 ++++++--
 7 files changed, 239 insertions(+), 289 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 5ed6a89..faf7e02 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -8,6 +8,8 @@ Trunk (Unreleased)
    FALCON-589 Add test cases for various feed operations on Hcat feeds (Karishma G 
    via Samarth Gupta)
   IMPROVEMENTS
+   FALCON-632 Refactoring, documentation stuff (Paul Isaychuk via Samarth Gupta)
+
    FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored
    (Paul Isaychuk via Samarth Gupta)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
index 1973bf8..0d89fac 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
@@ -69,28 +69,24 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
     String inputPath = pigTestDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private static final Logger logger = Logger.getLogger(EmbeddedPigScriptTest.class);
     private static final double TIMEOUT = 15;
+    String processName;
+    String process;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
-
         logger.info("in @BeforeClass");
+
         //copy pig script
         HadoopUtil.uploadDir(clusterFS, pigScriptDir, OSUtil.RESOURCES + "pig");
-
         Bundle bundle = BundleUtil.readELBundle();
         bundle.generateUniqueBundle();
         bundle = new Bundle(bundle, cluster);
-
         String startDate = "2010-01-02T00:40Z";
         String endDate = "2010-01-02T01:10Z";
-
         bundle.setInputFeedDataPath(inputPath);
         prefix = bundle.getFeedDataPathPrefix();
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
-
-        List<String> dataDates =
-            TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-
+        List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
     }
 
@@ -120,6 +116,8 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
         processElement.getWorkflow().setEngine(EngineType.PIG);
         bundles[0].setProcessData(processElement.toString());
         bundles[0].submitFeedsScheduleProcess(prism);
+        process = bundles[0].getProcessData();
+        processName = Util.readEntityName(process);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -129,70 +127,56 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
 
     @Test(groups = {"singleCluster"})
     public void getResumedProcessInstance() throws Exception {
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
-        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, bundles[0].getProcessData());
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        ServiceResponse status =
-            prism.getProcessHelper().getStatus(URLS.STATUS_URL, bundles[0].getProcessData());
+        ServiceResponse status = prism.getProcessHelper().getStatus(URLS.STATUS_URL, process);
         Assert.assertTrue(status.getMessage().contains("SUSPENDED"), "Process not suspended.");
-        prism.getProcessHelper().resume(URLS.RESUME_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().resume(URLS.RESUME_URL, process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
         InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
     @Test(groups = {"singleCluster"})
     public void getSuspendedProcessInstance() throws Exception {
-        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.SUSPENDED);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
         InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
     @Test(groups = {"singleCluster"})
     public void getRunningProcessInstance() throws Exception {
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
         InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
     @Test(groups = {"singleCluster"})
     public void getKilledProcessInstance() throws Exception {
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().delete(URLS.DELETE_URL, process);
         InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
 
     @Test(groups = {"singleCluster"})
     public void getSucceededProcessInstance() throws Exception {
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
         InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
-
         int counter = OSUtil.IS_WINDOWS ? 100 : 50;
         InstanceUtil.waitForBundleToReachState(cluster, Util.getProcessName(bundles[0]
             .getProcessData()), Job.Status.SUCCEEDED, counter);
-        r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING, processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
index ff227d6..acf3bb3 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
@@ -82,27 +82,29 @@ public class FeedInstanceStatusTest extends BaseTestClass {
         removeBundles();
     }
 
+    /**
+     * Goes through the whole feed replication workflow checking its instances status while
+     * submitting feed, scheduling it, performing different combinations of actions like
+     * -submit, -resume, -kill, -rerun.
+     */
     @Test(groups = {"multiCluster"})
     public void feedInstanceStatus_running() throws Exception {
         bundles[0].setInputFeedDataPath(feedInputPath);
 
         logger.info("cluster bundle1: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
-
-        ServiceResponse r = prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
-        Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
+        AssertUtil.assertSucceeded(prism.getClusterHelper()
+            .submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0)));
 
         logger.info("cluster bundle2: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
-        r = prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
-        Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
+        AssertUtil.assertSucceeded(prism.getClusterHelper()
+            .submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0)));
 
         logger.info("cluster bundle3: " + Util.prettyPrintXml(bundles[2].getClusters().get(0)));
-        r = prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[2].getClusters().get(0));
-        Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
+        AssertUtil.assertSucceeded(prism.getClusterHelper()
+            .submitEntity(URLS.SUBMIT_URL, bundles[2].getClusters().get(0)));
 
         String feed = bundles[0].getDataSets().get(0);
+        String feedName = Util.readEntityName(feed);
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2009-02-01T00:00Z", "2012-01-01T00:00Z"),
             XmlUtil.createRtention("hours(10)", ActionType.DELETE), null,
@@ -110,7 +112,7 @@ public class FeedInstanceStatusTest extends BaseTestClass {
         String startTime = TimeUtil.getTimeWrtSystemTime(-50);
 
         feed = InstanceUtil.setFeedCluster(feed, XmlUtil.createValidity(startTime,
-                TimeUtil.addMinsToTime(startTime, 65)),
+            TimeUtil.addMinsToTime(startTime, 65)),
             XmlUtil.createRtention("hours(10)", ActionType.DELETE),
             Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.SOURCE,
             "US/${cluster.colo}");
@@ -126,48 +128,38 @@ public class FeedInstanceStatusTest extends BaseTestClass {
             Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.SOURCE,
             "UK/${cluster.colo}");
 
-
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
         //status before submit
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 100) + "&end=" +
-                TimeUtil.addMinsToTime(startTime, 120));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 100)
+                + "&end=" + TimeUtil.addMinsToTime(startTime, 120));
 
         AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed));
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 100));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 100));
 
         AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
 
         // both replication instances
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 100));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 100));
 
         // single instance at -30
-        prism.getFeedHelper().getProcessInstanceStatus(Util.readEntityName(feed),
-            "?start=" + TimeUtil
-                .addMinsToTime(startTime, 20));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 20));
 
         //single at -10
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //single at 10
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //single at 30
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
 
         String postFix = "/US/" + cluster2.getClusterHelper().getColo();
         String prefix = bundles[0].getFeedDataPathPrefix();
@@ -180,102 +172,74 @@ public class FeedInstanceStatusTest extends BaseTestClass {
         HadoopUtil.lateDataReplenish(cluster3FS, 80, 20, prefix, postFix);
 
         // both replication instances
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 100));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 100));
 
         // single instance at -30
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 20));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 20));
 
         //single at -10
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //single at 10
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //single at 30
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
 
         logger.info("Wait till feed goes into running ");
 
         //suspend instances -10
-        prism.getFeedHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 40));
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 20) + "&end=" +
-                TimeUtil.addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceSuspend(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 20)
+                + "&end=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //resuspend -10 and suspend -30 source specific
-        prism.getFeedHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(feed),
-                "?start=" + TimeUtil
-                    .addMinsToTime(startTime, 20) + "&end=" +
-                    TimeUtil.addMinsToTime(startTime, 40));
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 20) + "&end=" +
-                TimeUtil.addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceSuspend(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 20)
+                + "&end=" + TimeUtil.addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 20)
+                + "&end=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //resume -10 and -30
-        prism.getFeedHelper()
-            .getProcessInstanceResume(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 20) + "&end=" +
-                TimeUtil.addMinsToTime(startTime, 40));
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 20) + "&end=" +
-                TimeUtil.addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceResume(feedName, "?start=" + TimeUtil
+            .addMinsToTime(startTime, 20) + "&end=" + TimeUtil.addMinsToTime(startTime, 40));
+        prism.getFeedHelper().getProcessInstanceStatus(feedName, "?start=" + TimeUtil
+            .addMinsToTime(startTime, 20) + "&end=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //get running instances
-        prism.getFeedHelper().getRunningInstance(URLS.INSTANCE_RUNNING, Util.readEntityName(feed));
+        prism.getFeedHelper().getRunningInstance(URLS.INSTANCE_RUNNING, feedName);
 
         //rerun succeeded instance
-        prism.getFeedHelper()
-            .getProcessInstanceRerun(Util.readEntityName(feed), "?start=" + startTime);
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 20));
+        prism.getFeedHelper().getProcessInstanceRerun(feedName, "?start=" + startTime);
+        prism.getFeedHelper().getProcessInstanceStatus(feedName, "?start=" + startTime
+            + "&end=" + TimeUtil.addMinsToTime(startTime, 20));
 
         //kill instance
-        prism.getFeedHelper()
-            .getProcessInstanceKill(Util.readEntityName(feed), "?start=" + TimeUtil
-                .addMinsToTime(startTime, 44));
-        prism.getFeedHelper()
-            .getProcessInstanceKill(Util.readEntityName(feed), "?start=" + startTime);
+        prism.getFeedHelper().getProcessInstanceKill(feedName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 44));
+        prism.getFeedHelper().getProcessInstanceKill(feedName, "?start=" + startTime);
 
         //end time should be less than end of validity i.e startTime + 110
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 110));
-
+        prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 110));
 
         //rerun killed instance
-        prism.getFeedHelper()
-            .getProcessInstanceRerun(Util.readEntityName(feed), "?start=" + startTime);
-        prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 110));
+        prism.getFeedHelper().getProcessInstanceRerun(feedName, "?start=" + startTime);
+        prism.getFeedHelper().getProcessInstanceStatus(feedName, "?start=" + startTime
+            + "&end=" + TimeUtil.addMinsToTime(startTime, 110));
 
         //kill feed
         prism.getFeedHelper().delete(URLS.DELETE_URL, feed);
-        InstancesResult responseInstance = prism.getFeedHelper()
-            .getProcessInstanceStatus(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil
-                    .addMinsToTime(startTime, 110));
+        InstancesResult responseInstance = prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 110));
 
         logger.info(responseInstance.getMessage());
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
index e7b2616..e4129e6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
@@ -55,29 +55,25 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     private String testDir = "/ProcessInstanceKillsTest";
     private String baseTestHDFSDir = baseHDFSDir + testDir;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
-    private String feedInputPath = baseTestHDFSDir +
-        "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + "/input" + datePattern;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + datePattern;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceKillsTest.class);
     private static final double TIMEOUT = 15;
+    String processName;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
         LOGGER.info("in @BeforeClass");
         HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-
         Bundle b = BundleUtil.readELBundle();
         b.generateUniqueBundle();
         b = new Bundle(b, cluster);
-
         String startDate = "2010-01-01T23:20Z";
         String endDate = "2010-01-02T01:21Z";
-
         b.setInputFeedDataPath(feedInputPath);
         String prefix = b.getFeedDataPathPrefix();
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
     }
@@ -85,12 +81,12 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     @BeforeMethod(alwaysRun = true)
     public void setup(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
-
         bundles[0] = BundleUtil.readELBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         bundles[0].setInputFeedDataPath(feedInputPath);
+        processName = Util.readEntityName(bundles[0].getProcessData());
     }
 
     @AfterMethod(alwaysRun = true)
@@ -115,8 +111,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.KILLED);
     }
 
@@ -139,8 +134,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T00:03Z&end=2010-01-02T00:03Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T00:03Z&end=2010-01-02T00:03Z");
         InstanceUtil.validateResponse(r, 1, 0, 0, 0, 1);
     }
 
@@ -162,8 +156,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T00:03Z&end=2010-01-02T00:30Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T00:03Z&end=2010-01-02T00:30Z");
         InstanceUtil.validateResponse(r, 3, 0, 0, 0, 3);
         LOGGER.info(r.toString());
     }
@@ -196,9 +189,8 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         String startTimeRequest = TimeUtil.getTimeWrtSystemTime(-17);
         String endTimeRequest = TimeUtil.getTimeWrtSystemTime(23);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=" + startTimeRequest + "&end=" + endTimeRequest);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
+            "?start=" + startTimeRequest + "&end=" + endTimeRequest);
         LOGGER.info(r.toString());
     }
 
@@ -222,8 +214,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         String startTime = TimeUtil.getTimeWrtSystemTime(1);
         String endTime = TimeUtil.getTimeWrtSystemTime(40);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
+            .getProcessInstanceKill(processName, "?start=" + startTime + "&end=" + endTime);
         LOGGER.info(r.getMessage());
         Assert.assertEquals(r.getInstances(), null);
     }
@@ -245,12 +236,10 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:15Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T01:05Z&end=2010-01-02T01:15Z");
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateResponse(result, 5, 2, 0, 0, 3);
     }
 
@@ -269,13 +258,10 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:20Z");
+        prism.getProcessHelper().getProcessInstanceKill(processName, "?start=2010-01-02T01:20Z");
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateResponse(result, 5, 4, 0, 0, 1);
     }
 
@@ -294,11 +280,9 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
         prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+            .getProcessInstanceSuspend(processName, "?start=2010-01-02T01:00Z");
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.KILLED);
     }
 
@@ -316,15 +300,13 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstanceReachState(serverOC.get(0), Util.getProcessName(bundles[0]
-            .getProcessData()), 1, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
+        InstanceUtil.waitTillInstanceReachState(serverOC.get(0), processName, 1,
+            CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.SUCCEEDED);
     }
 
-
     @AfterClass(alwaysRun = true)
     public void deleteData() throws Exception {
         LOGGER.info("in @AfterClass");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
index 119d871..df65a79 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
@@ -53,30 +53,28 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
 
     private String baseTestDir = baseHDFSDir + "/ProcessInstanceRerunTest";
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    private String feedInputPath = baseTestDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath = baseTestDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedInputTimedOutPath =
-        baseTestDir + "/timedout/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-
+    private String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestDir + "/input" + datePattern;
+    private String feedOutputPath = baseTestDir + "/output-data" + datePattern;
+    private String feedInputTimedOutPath = baseTestDir + "/timedout" + datePattern;
     private ColoHelper cluster = servers.get(0);
     private FileSystem clusterFS = serverFS.get(0);
     private OozieClient clusterOC = serverOC.get(0);
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceRerunTest.class);
     private static final double TIMEOUT = 10;
+    private String processName;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
         LOGGER.info("in @BeforeClass");
         HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
         Bundle b = BundleUtil.readELBundle();
-
         b = new Bundle(b, cluster);
         String startDate = "2010-01-02T00:40Z";
         String endDate = "2010-01-02T01:20Z";
         b.setInputFeedDataPath(feedInputPath);
         String prefix = b.getFeedDataPathPrefix();
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
     }
@@ -89,6 +87,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].generateUniqueBundle();
         bundles[0].setInputFeedDataPath(feedInputPath);
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        processName = bundles[0].getProcessName();
     }
 
     @AfterMethod(alwaysRun = true)
@@ -111,17 +110,15 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
+        String process = bundles[0].getProcessData();
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:16Z");
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:16Z");
         InstanceUtil.validateResponse(r, 4, 0, 0, 0, 4);
-        List<String> wfIDs =
-            InstanceUtil.getWorkflows(cluster, Util.getProcessName(bundles[0].getProcessData()));
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+        List<String> wfIDs = InstanceUtil.getWorkflows(cluster, processName);
+        prism.getProcessHelper().getProcessInstanceRerun(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 6, 5, 1, 0);
     }
 
@@ -137,18 +134,16 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
-        LOGGER.info("process: " + Util.prettyPrintXml(bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        LOGGER.info("process: " + Util.prettyPrintXml(process));
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.validateResponse(r, 3, 0, 0, 0, 3);
-        List<String> wfIDs =
-            InstanceUtil.getWorkflows(cluster, Util.getProcessName(bundles[0].getProcessData()));
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+        List<String> wfIDs =  InstanceUtil.getWorkflows(cluster, processName);
+        prism.getProcessHelper().
+            getProcessInstanceRerun(processName, "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 3, 3, 0, 0);
     }
 
@@ -166,16 +161,14 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
+        String process = bundles[0].getProcessData();
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.validateResponse(r, 3, 0, 0, 0, 3);
-        List<String> wfIDs =
-            InstanceUtil.getWorkflows(cluster, Util.getProcessName(bundles[0].getProcessData()));
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+        List<String> wfIDs = InstanceUtil.getWorkflows(cluster, processName);
+        prism.getProcessHelper().getProcessInstanceRerun(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         TimeUtil.sleepSeconds(TIMEOUT);
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 6, 6, 0, 0);
     }
@@ -193,15 +186,11 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
-        prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
-        String wfID = InstanceUtil.getWorkflows(cluster,
-            Util.getProcessName(bundles[0].getProcessData()), Status.KILLED).get(0);
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        String process = bundles[0].getProcessData();
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
+        prism.getProcessHelper().getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
+        String wfID = InstanceUtil.getWorkflows(cluster, processName, Status.KILLED).get(0);
+        prism.getProcessHelper().getProcessInstanceRerun(processName, "?start=2010-01-02T01:00Z");
         Assert.assertTrue(InstanceUtil.isWorkflowRunning(clusterOC, wfID));
     }
 
@@ -219,15 +208,13 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
-        String wfID = InstanceUtil.getWorkflows(cluster, Util.getProcessName(bundles[0]
-            .getProcessData()), Status.RUNNING, Status.SUCCEEDED).get(0);
-        InstanceUtil.waitTillInstanceReachState(clusterOC, Util.readEntityName(bundles[0]
-            .getProcessData()), 0, CoordinatorAction
+        String process = bundles[0].getProcessData();
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
+        String wfID = InstanceUtil.getWorkflows(cluster, processName, Status.RUNNING,
+               Status.SUCCEEDED).get(0);
+        InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 0, CoordinatorAction
             .Status.SUCCEEDED, EntityType.PROCESS);
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        prism.getProcessHelper().getProcessInstanceRerun(processName, "?start=2010-01-02T01:00Z");
         Assert.assertTrue(InstanceUtil.isWorkflowRunning(clusterOC, wfID));
     }
 
@@ -245,14 +232,11 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(2);
         bundles[0].submitFeedsScheduleProcess(prism);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:06Z");
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:06Z");
-        Assert.assertEquals(InstanceUtil
-            .getInstanceStatus(cluster, Util.getProcessName(bundles[0].getProcessData()), 0, 1),
+        prism.getProcessHelper().getProcessInstanceSuspend(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:06Z");
+        prism.getProcessHelper().getProcessInstanceRerun(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:06Z");
+        Assert.assertEquals(InstanceUtil.getInstanceStatus(cluster, processName, 0, 1),
             CoordinatorAction.Status.SUSPENDED);
     }
 
@@ -269,14 +253,13 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
-        InstanceUtil.waitTillInstanceReachState(clusterOC, Util.readEntityName(bundles[0]
-            .getProcessData()), 2, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
-        List<String> wfIDs =
-            InstanceUtil.getWorkflows(cluster, Util.getProcessName(bundles[0].getProcessData()));
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+        String process = bundles[0].getProcessData();
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
+        InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 2,
+            CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
+        List<String> wfIDs = InstanceUtil.getWorkflows(cluster, processName);
+        prism.getProcessHelper().getProcessInstanceRerun(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 3, 3, 0, 0);
     }
 
@@ -297,13 +280,11 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
         CoordinatorAction.Status s;
-        InstanceUtil.waitTillInstanceReachState(clusterOC, Util.getProcessName(bundles[0]
-            .getProcessData()), 1, CoordinatorAction.Status.TIMEDOUT, EntityType.PROCESS);
-        prism.getProcessHelper()
-            .getProcessInstanceRerun(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
-        s = InstanceUtil
-            .getInstanceStatus(cluster, Util.readEntityName(bundles[0].getProcessData()), 0, 0);
+        InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 1,
+            CoordinatorAction.Status.TIMEDOUT, EntityType.PROCESS);
+        prism.getProcessHelper().getProcessInstanceRerun(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+        s = InstanceUtil.getInstanceStatus(cluster, processName, 0, 0);
         Assert.assertEquals(s, CoordinatorAction.Status.WAITING,
             "instance should have been in WAITING state");
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
index fc8e4a8..7647d15 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
@@ -52,45 +52,43 @@ public class ProcessLibPathTest extends BaseTestClass {
     String testDir = baseHDFSDir + "/ProcessLibPath";
     String testLibDir = testDir + "/TestLib";
     private static final Logger logger = Logger.getLogger(ProcessLibPathTest.class);
+    String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String processName;
+    String process;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
-
         logger.info("in @BeforeClass");
+
         //common lib for both test cases
         HadoopUtil.uploadDir(clusterFS, testLibDir, OSUtil.RESOURCES_OOZIE + "lib");
-
         Bundle b = BundleUtil.readELBundle();
         b.generateUniqueBundle();
         b = new Bundle(b, cluster);
-
         String startDate = "2010-01-01T22:00Z";
         String endDate = "2010-01-02T03:00Z";
-
-        b.setInputFeedDataPath(testDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        b.setInputFeedDataPath(testDir + "/input" + datePattern);
         String prefix = b.getFeedDataPathPrefix();
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
     }
 
-
     @BeforeMethod(alwaysRun = true)
     public void testName(Method method) throws Exception {
         logger.info("test name: " + method.getName());
         bundles[0] = BundleUtil.readELBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
-        bundles[0].setInputFeedDataPath(baseHDFSDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setInputFeedDataPath(baseHDFSDir + datePattern);
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(
-            baseHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setOutputFeedLocationData(baseHDFSDir + "/output-data" + datePattern);
         bundles[0].setProcessConcurrency(1);
         bundles[0].setProcessLibPath(testLibDir);
+        process = bundles[0].getProcessData();
+        processName = Util.readEntityName(process);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -109,13 +107,11 @@ public class ProcessLibPathTest extends BaseTestClass {
         HadoopUtil.uploadDir(clusterFS, workflowDir, OSUtil.RESOURCES_OOZIE);
         HadoopUtil.deleteDirIfExists(workflowDir + "/lib", clusterFS);
         bundles[0].setProcessWorkflow(workflowDir);
-        logger.info("processData: " + Util.prettyPrintXml(bundles[0].getProcessData()));
+        logger.info("processData: " + Util.prettyPrintXml(process));
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
-        OozieUtil.createMissingDependencies(cluster, EntityType.PROCESS, Util.readEntityName(bundles[0]
-                .getProcessData()), 0);
-        InstanceUtil
-            .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
+        OozieUtil.createMissingDependencies(cluster, EntityType.PROCESS, processName, 0);
+        InstanceUtil.waitForBundleToReachState(cluster, processName, Status.SUCCEEDED);
     }
 
     /**
@@ -131,12 +127,10 @@ public class ProcessLibPathTest extends BaseTestClass {
         HadoopUtil.copyDataToFolder(clusterFS, workflowDir + "/lib",
             OSUtil.RESOURCES + "ivory-oozie-lib-0.1.jar");
         bundles[0].setProcessWorkflow(workflowDir);
-        logger.info("processData: " + Util.prettyPrintXml(bundles[0].getProcessData()));
+        logger.info("processData: " + Util.prettyPrintXml(process));
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster, bundles[0].getProcessData(), 0);
-        OozieUtil.createMissingDependencies(cluster, EntityType.PROCESS, Util.readEntityName(bundles[0]
-                .getProcessData()), 0);
-        InstanceUtil
-            .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
+        InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
+        OozieUtil.createMissingDependencies(cluster, EntityType.PROCESS, processName, 0);
+        InstanceUtil.waitForBundleToReachState(cluster, processName, Status.SUCCEEDED);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5dfe5cde/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
index 85bd770..1d900d9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
@@ -65,7 +65,6 @@ public class RetentionTest extends BaseTestClass {
     String baseTestHDFSDir = baseHDFSDir + "/RetentionTest/";
     String testHDFSDir = baseTestHDFSDir + TEST_FOLDERS;
     private static final Logger logger = Logger.getLogger(RetentionTest.class);
-
     ColoHelper cluster = servers.get(0);
     FileSystem clusterFS = serverFS.get(0);
     OozieClient clusterOC = serverOC.get(0);
@@ -85,12 +84,26 @@ public class RetentionTest extends BaseTestClass {
         removeBundles();
     }
 
+    /**
+     * Particular test case for https://issues.apache.org/jira/browse/FALCON-321
+     * @throws Exception
+     */
     @Test
     public void testRetentionWithEmptyDirectories() throws Exception {
-        // test for https://issues.apache.org/jira/browse/FALCON-321
         testRetention(24, RetentionUnit.HOURS, true, FeedType.DAILY, false);
     }
 
+    /**
+     * Tests retention with different parameters. Validates its results based on expected and
+     * actual retained data.
+     *
+     * @param retentionPeriod period for which data should be retained
+     * @param retentionUnit type of retention limit attribute
+     * @param gaps defines gaps within list of data folders
+     * @param feedType feed type
+     * @param withData should folders be filled with data or not
+     * @throws Exception
+     */
     @Test(groups = {"0.1", "0.2", "prism"}, dataProvider = "betterDP", priority = -1)
     public void testRetention(final int retentionPeriod, final RetentionUnit retentionUnit,
         final boolean gaps, final FeedType feedType, final boolean withData) throws Exception {
@@ -125,16 +138,28 @@ public class RetentionTest extends BaseTestClass {
         if (gap) {
             skip = gaps[new Random().nextInt(gaps.length)];
         }
-
         final DateTime today = new DateTime(DateTimeZone.UTC);
         final List<DateTime> times = TimeUtil.getDatesOnEitherSide(
             feedType.addTime(today, -36), feedType.addTime(today, 36), skip, feedType);
         final List<String> dataDates = TimeUtil.convertDatesToString(times, feedType.getFormatter());
         logger.info("dataDates = " + dataDates);
-
         HadoopUtil.replenishData(clusterFS, testHDFSDir, dataDates, withData);
     }
 
+    /**
+     * Schedules feed and waits till retention succeeds. Makes validation of data which was removed
+     * and which was retained.
+     *
+     * @param feed analyzed retention feed
+     * @param feedType feed type
+     * @param retentionUnit type of retention limit attribute
+     * @param retentionPeriod period for which data should be retained
+     * @throws OozieClientException
+     * @throws IOException
+     * @throws URISyntaxException
+     * @throws AuthenticationException
+     * @throws JMSException
+     */
     private void commonDataRetentionWorkflow(String feed, FeedType feedType,
         RetentionUnit retentionUnit, int retentionPeriod) throws OozieClientException,
         IOException, URISyntaxException, AuthenticationException, JMSException {
@@ -148,22 +173,20 @@ public class RetentionTest extends BaseTestClass {
         JmsMessageConsumer messageConsumer = new JmsMessageConsumer("FALCON." + feedName,
                 cluster.getClusterHelper().getActiveMQ());
         messageConsumer.start();
-
         final DateTime currentTime = new DateTime(DateTimeZone.UTC);
         String bundleId = OozieUtil.getBundles(clusterOC, feedName, EntityType.FEED).get(0);
 
         List<String> workflows = OozieUtil.waitForRetentionWorkflowToSucceed(bundleId, clusterOC);
         logger.info("workflows: " + workflows);
-
         messageConsumer.interrupt();
         Util.printMessageData(messageConsumer);
+
         //now look for cluster data
         List<String> finalData = Util.getHadoopDataFromDir(clusterFS, feed, testHDFSDir);
 
         //now see if retention value was matched to as expected
         List<String> expectedOutput = filterDataOnRetention(initialData, currentTime, retentionUnit,
             retentionPeriod, feedType);
-
         logger.info("initialData = " + initialData);
         logger.info("finalData = " + finalData);
         logger.info("expectedOutput = " + expectedOutput);
@@ -171,23 +194,31 @@ public class RetentionTest extends BaseTestClass {
         final List<String> missingData = new ArrayList<String>(initialData);
         missingData.removeAll(expectedOutput);
         validateDataFromFeedQueue(feedName, messageConsumer.getReceivedMessages(), missingData);
-
         Assert.assertEquals(finalData.size(), expectedOutput.size(),
-            "sizes of outputs are different! please check");
+            "Expected and actual sizes of retained data are different! Please check.");
 
         Assert.assertTrue(Arrays.deepEquals(finalData.toArray(new String[finalData.size()]),
             expectedOutput.toArray(new String[expectedOutput.size()])));
     }
 
+    /**
+     * Makes validation based on comparison of data which is expected to be removed with data
+     * mentioned in messages from ActiveMQ
+     *
+     * @param feedName feed name
+     * @param messages messages from ActiveMQ
+     * @param missingData data which is expected to be removed after retention succeeded
+     * @throws OozieClientException
+     * @throws JMSException
+     */
     private void validateDataFromFeedQueue(String feedName, List<MapMessage> messages,
         List<String> missingData) throws OozieClientException, JMSException {
         //just verify that each element in queue is same as deleted data!
         List<String> workflowIds = OozieUtil.getWorkflowJobs(cluster,
                 OozieUtil.getBundles(clusterOC, feedName, EntityType.FEED).get(0));
 
-        //create queuedata folderList:
+        //create queue data folderList:
         List<String> deletedFolders = new ArrayList<String>();
-
         for (MapMessage message : messages) {
             if (message != null) {
                 Assert.assertEquals(message.getString("entityName"), feedName);
@@ -205,7 +236,6 @@ public class RetentionTest extends BaseTestClass {
                     cluster.getFeedHelper().getActiveMQ());
             }
         }
-
         Assert.assertEquals(deletedFolders.size(), missingData.size(),
             "Output size is different than expected!");
         Assert.assertTrue(Arrays.deepEquals(missingData.toArray(new String[missingData.size()]),
@@ -213,6 +243,16 @@ public class RetentionTest extends BaseTestClass {
             "The missing data and message for delete operation don't correspond");
     }
 
+    /**
+     * Evaluates amount of data which is expected to be retained
+     *
+     * @param inputData initial data on cluster
+     * @param currentTime current date
+     * @param retentionUnit type of retention limit attribute
+     * @param retentionPeriod period for which data should be retained
+     * @param feedType feed type
+     * @return list of data folders which are expected to be present on cluster
+     */
     private List<String> filterDataOnRetention(List<String> inputData, DateTime currentTime,
         RetentionUnit retentionUnit, int retentionPeriod, FeedType feedType) {
         final List<String> finalData = new ArrayList<String>();
@@ -232,6 +272,9 @@ public class RetentionTest extends BaseTestClass {
 
     final static int[] gaps = new int[]{2, 4, 5, 1};
 
+    /**
+     * Provides different sets of parameters for retention workflow.
+     */
     @DataProvider(name = "betterDP")
     public Object[][] getTestData(Method m) {
         // a negative value like -4 should be covered in validation scenarios.


[40/41] git commit: FALCON-675 Request URLS moved from parameters into methods in falcon-regression(2). Fixing missed bug

Posted by ra...@apache.org.
FALCON-675 Request URLS moved from parameters into methods in falcon-regression(2). Fixing missed bug


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

Branch: refs/heads/FALCON-585
Commit: 5766b74ae574bd8657d19ac11ab66d1293f2a0af
Parents: 051a3d2
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Mon Sep 8 16:28:46 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Mon Sep 8 16:29:34 2014 +0300

----------------------------------------------------------------------
 .../java/org/apache/falcon/regression/AuthorizationTest.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5766b74a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
index b309ea7..4636416 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
@@ -664,13 +664,13 @@ public class AuthorizationTest extends BaseTestClass {
         bundles[0].submitClusters(prism);
         bundles[0].submitFeeds(prism);
         //schedule input feed by U1
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
 
         //by U2 schedule process dependent on scheduled feed by U1
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        ServiceResponse serviceResponse = prism.getProcessHelper().submitAndSchedule(Util
-                 .URLS.SUBMIT_AND_SCHEDULE_URL, process, MerlinConstants.USER2_NAME);
+        ServiceResponse serviceResponse = prism.getProcessHelper().submitAndSchedule(process,
+            MerlinConstants.USER2_NAME);
         AssertUtil.assertSucceeded(serviceResponse);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
 


[18/41] git commit: FALCON-649 Remove unnecessary validation for Instance start time in FalconCLI. Contributed by Balu Vellanki

Posted by ra...@apache.org.
FALCON-649 Remove unnecessary validation for Instance start time in FalconCLI. Contributed by Balu Vellanki


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

Branch: refs/heads/FALCON-585
Commit: 8f30ae0e0a47288e1a924ac315c377103a883927
Parents: 0bd9c77
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Tue Sep 2 13:11:03 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Tue Sep 2 13:11:03 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                               |  3 +++
 client/src/main/java/org/apache/falcon/cli/FalconCLI.java | 10 ++--------
 2 files changed, 5 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/8f30ae0e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ca12d59..45f0ac3 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -72,6 +72,9 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-649 Remove unnecessary validation for Instance start time in
+   FalconCLI (Balu Vellanki via Venkatesh Seetharam)
+
    FALCON-579 Lineage breaks if feed.xml doesn't have the date pattern in
    feed path location (Sowmya Ramesh via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/8f30ae0e/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
index f5b30f0..a42da13 100644
--- a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
+++ b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
@@ -235,7 +235,7 @@ public class FalconCLI {
 
         colo = getColo(colo);
         String instanceAction = "instance";
-        validateInstanceCommands(optionsList, entity, type, start, colo);
+        validateInstanceCommands(optionsList, entity, type, colo);
 
 
         if (optionsList.contains(RUNNING_OPT)) {
@@ -290,7 +290,7 @@ public class FalconCLI {
 
     private void validateInstanceCommands(Set<String> optionsList,
                                           String entity, String type,
-                                          String start, String colo) throws FalconCLIException {
+                                          String colo) throws FalconCLIException {
 
         if (StringUtils.isEmpty(entity)) {
             throw new FalconCLIException("Missing argument: name");
@@ -304,12 +304,6 @@ public class FalconCLI {
             throw new FalconCLIException("Missing argument: colo");
         }
 
-        if (!optionsList.contains(RUNNING_OPT)) {
-            if (StringUtils.isEmpty(start)) {
-                throw new FalconCLIException("Missing argument: start");
-            }
-        }
-
         if (optionsList.contains(CLUSTERS_OPT)) {
             if (optionsList.contains(RUNNING_OPT)
                     || optionsList.contains(LOG_OPT)


[12/41] git commit: FALCON-646 Refactoring, documentation stuff. Contributed by Paul Isaychuk

Posted by ra...@apache.org.
FALCON-646 Refactoring, documentation stuff. Contributed by Paul Isaychuk


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

Branch: refs/heads/FALCON-585
Commit: 9774414a6e654edf6071d70d630bcbb0a960b420
Parents: 23eed9f
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Fri Aug 29 13:12:19 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Fri Aug 29 13:12:19 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   6 +-
 .../falcon/regression/core/util/Util.java       |  82 ++++++-----
 .../regression/FeedSubmitAndScheduleTest.java   |  32 ++---
 .../regression/prism/FeedRetentionTest.java     |  39 ++---
 .../regression/prism/OptionalInputTest.java     | 143 ++++++-------------
 5 files changed, 123 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9774414a/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index ad6ef79..f5cfb93 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,8 +9,10 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
-   
-   FALCON-572 HadoopUtil cleanup in falcon-regression (Ruslan Ostafiychuk via Samarth Gupta)  
+
+   FALCON-646 Refactoring, documentation stuff (Paul Isaychuk via Ruslan Ostafiychuk)
+
+   FALCON-572 HadoopUtil cleanup in falcon-regression (Ruslan Ostafiychuk via Samarth Gupta)
    FALCON-632 Refactoring, documentation stuff (Paul Isaychuk via Samarth Gupta)
 
    FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9774414a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
index 3fc9388..6485784 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
@@ -119,12 +119,10 @@ public final class Util {
     }
 
     public static APIResult parseResponse(ServiceResponse response) throws JAXBException {
-
         if (!isXML(response.getMessage())) {
             return new APIResult(APIResult.Status.FAILED, response.getMessage(), "somerandomstring",
                 response.getCode());
         }
-
         JAXBContext jc = JAXBContext.newInstance(APIResult.class);
         Unmarshaller u = jc.createUnmarshaller();
         APIResult temp;
@@ -143,7 +141,6 @@ public final class Util {
                 temp.setStatus(APIResult.Status.FAILED);
             }
         }
-
         return temp;
     }
 
@@ -170,30 +167,25 @@ public final class Util {
     }
 
     public static String getUniqueString() {
-
         return "-" + UUID.randomUUID().toString().split("-")[0];
     }
 
     public static List<String> getHadoopDataFromDir(FileSystem fs, String feed, String dir)
         throws IOException {
         List<String> finalResult = new ArrayList<String>();
-
         String feedPath = getFeedPath(feed);
         int depth = feedPath.split(dir)[1].split("/").length - 1;
         List<Path> results = HadoopUtil.getAllDirsRecursivelyHDFS(fs,
             new Path(dir), depth);
-
         for (Path result : results) {
             int pathDepth = result.toString().split(dir)[1].split("/").length - 1;
             if (pathDepth == depth) {
                 finalResult.add(result.toString().split(dir)[1]);
             }
         }
-
         return finalResult;
     }
 
-
     public static String setFeedProperty(String feed, String propertyName, String propertyValue) {
         FeedMerlin feedObject = new FeedMerlin(feed);
         boolean found = false;
@@ -205,20 +197,15 @@ public final class Util {
                 break;
             }
         }
-
         if (!found) {
             Property property = new Property();
             property.setName(propertyName);
             property.setValue(propertyValue);
             feedObject.getProperties().getProperties().add(property);
         }
-
-
         return feedObject.toString();
-
     }
 
-
     public static String getFeedPath(String feed) {
         FeedMerlin feedObject = new FeedMerlin(feed);
         for (Location location : feedObject.getLocations().getLocations()) {
@@ -236,7 +223,6 @@ public final class Util {
         return feedObject.toString();
     }
 
-
     public static String setFeedPathValue(String feed, String pathValue) {
         FeedMerlin feedObject = new FeedMerlin(feed);
         for (Location location : feedObject.getLocations().getLocations()) {
@@ -247,11 +233,9 @@ public final class Util {
         return feedObject.toString();
     }
 
-
     public static String findFolderBetweenGivenTimeStamps(DateTime startTime, DateTime endTime,
                                                           List<String> folderList) {
         DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy/MM/dd/HH/mm");
-
         for (String folder : folderList) {
             if (folder.compareTo(formatter.print(startTime)) >= 0
                     &&
@@ -285,8 +269,7 @@ public final class Util {
                 .getQaHost(), coloHelper.getProcessHelper().getUsername(),
             coloHelper.getProcessHelper().getPassword(),
             "cat /var/log/ivory/application.* | grep \"" + workflowId + "\" | grep "
-                    +
-                "\"Received\" | awk '{print $2}'",
+                    + "\"Received\" | awk '{print $2}'",
             coloHelper.getProcessHelper().getUsername(),
             coloHelper.getProcessHelper().getIdentityFile()
         );
@@ -313,7 +296,6 @@ public final class Util {
         for (String line : raw) {
             finalList.add(line.split(",")[0]);
         }
-
         return finalList;
     }
 
@@ -327,7 +309,6 @@ public final class Util {
 
     public static void startService(IEntityManagerHelper helper)
         throws IOException, JSchException, AuthenticationException, URISyntaxException {
-
         ExecUtil.runRemoteScriptAsSudo(helper.getQaHost(), helper.getUsername(),
             helper.getPassword(), helper.getServiceStartCmd(), helper.getServiceUser(),
             helper.getIdentityFile());
@@ -372,17 +353,13 @@ public final class Util {
         }
     }
 
-
     public static String getEnvClusterXML(String cluster, String prefix) {
-
-        ClusterMerlin clusterObject =
-            getClusterObject(cluster);
+        ClusterMerlin clusterObject = getClusterObject(cluster);
         if ((null == prefix) || prefix.isEmpty()) {
             prefix = "";
         } else {
             prefix = prefix + ".";
         }
-
         String hcatEndpoint = Config.getProperty(prefix + "hcat_endpoint");
 
         //now read and set relevant values
@@ -401,7 +378,6 @@ public final class Util {
                 iface.setEndpoint(hcatEndpoint);
             }
         }
-
         //set colo name:
         clusterObject.setColo(Config.getProperty(prefix + "colo"));
         // properties in the cluster needed when secure mode is on
@@ -454,6 +430,13 @@ public final class Util {
         return null;
     }
 
+    /**
+     * Compares two definitions
+     * @param server1 server where 1st definition is stored
+     * @param server2 server where 2nd definition is stored
+     * @param entity entity which is under analysis
+     * @return are definitions identical
+     */
     public static boolean isDefinitionSame(ColoHelper server1, ColoHelper server2,
                                            String entity)
         throws URISyntaxException, IOException, AuthenticationException, JAXBException,
@@ -463,10 +446,9 @@ public final class Util {
     }
 
     /**
-     * emuns used for instance api.
+     * enums used for instance api.
      */
     public enum URLS {
-
         LIST_URL("/api/entities/list"),
         SUBMIT_URL("/api/entities/submit"),
         GET_ENTITY_DEFINITION("/api/entities/definition"),
@@ -497,17 +479,27 @@ public final class Util {
         }
     }
 
-
+    /**
+     * @param pathString whole path
+     * @return path to basic data folder
+     */
     public static String getPathPrefix(String pathString) {
         return pathString.substring(0, pathString.indexOf('$'));
     }
 
+    /**
+     * @param path whole path
+     * @return file name which is retrieved from a path
+     */
     public static String getFileNameFromPath(String path) {
-
         return path.substring(path.lastIndexOf('/') + 1, path.length());
     }
 
-
+    /**
+     * Defines request type according to request url
+     * @param url request url
+     * @return request type
+     */
     public static String getMethodType(String url) {
         List<String> postList = new ArrayList<String>();
         postList.add("/entities/validate");
@@ -531,10 +523,14 @@ public final class Util {
                 return "delete";
             }
         }
-
         return "get";
     }
 
+    /**
+     * Prints xml in readable form
+     * @param xmlString xmlString
+     * @return formatted xmlString
+     */
     public static String prettyPrintXml(final String xmlString) {
         if (xmlString == null) {
             return null;
@@ -554,19 +550,27 @@ public final class Util {
         } catch (TransformerException e) {
             return xmlString;
         }
-
     }
 
+    /**
+     * Converts json string to readable form
+     * @param jsonString json string
+     * @return formatted string
+     */
     public static String prettyPrintJson(final String jsonString) {
         if (jsonString == null) {
             return null;
         }
         Gson gson = new GsonBuilder().setPrettyPrinting().create();
         JsonElement json = new JsonParser().parse(jsonString);
-
         return gson.toJson(json);
     }
 
+    /**
+     * Prints xml or json in pretty and readable format
+     * @param str xml or json string
+     * @return converted xml or json
+     */
     public static String prettyPrintXmlOrJson(final String str) {
         if (str == null) {
             return null;
@@ -583,6 +587,13 @@ public final class Util {
         return str;
     }
 
+    /**
+     * Tries to get entity definition.
+     * @param cluster cluster where definition is stored
+     * @param entity entity for which definition is required
+     * @param shouldReturn should the definition be successfully retrieved or not
+     * @return entity definition
+     */
     public static String getEntityDefinition(ColoHelper cluster,
                                              String entity,
                                              boolean shouldReturn) throws
@@ -597,10 +608,8 @@ public final class Util {
         } else {
             helper = cluster.getClusterHelper();
         }
-
         ServiceResponse response = helper.getEntityDefinition(URLS
             .GET_ENTITY_DEFINITION, entity);
-
         if (shouldReturn) {
             AssertUtil.assertSucceeded(response);
         } else {
@@ -608,7 +617,6 @@ public final class Util {
         }
         String result = response.getMessage();
         Assert.assertNotNull(result);
-
         return result;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9774414a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
index ab13dd4..38cf080 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
@@ -54,6 +54,7 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
     private OozieClient clusterOC = serverOC.get(0);
     private String aggregateWorkflowDir = baseHDFSDir + "/FeedSubmitAndScheduleTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedSubmitAndScheduleTest.class);
+    private String feed;
 
     @BeforeMethod(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
@@ -67,6 +68,7 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        feed = bundles[0].getDataSets().get(0);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -91,8 +93,8 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
         throws JAXBException, IOException, URISyntaxException, AuthenticationException {
         Assert.assertEquals(Util.parseResponse(prism.getClusterHelper()
             .submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0))).getStatusCode(), 200);
-        ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0));
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(URLS
+            .SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(response);
     }
 
@@ -109,17 +111,16 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
 
         //get created bundle id
         String bundleId = InstanceUtil
-            .getLatestBundleID(cluster, Util.readEntityName(bundles[0].getDataSets().get(0)),
-                EntityType.FEED);
+            .getLatestBundleID(cluster, Util.readEntityName(feed), EntityType.FEED);
 
         //try to submit and schedule the same process again
         ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
         //check that new bundle wasn't created
-        OozieUtil.verifyNewBundleCreation(cluster, bundleId, null, bundles[0].getDataSets().get(0), false, false);
+        OozieUtil.verifyNewBundleCreation(cluster, bundleId, null, feed, false, false);
     }
 
     /**
@@ -131,7 +132,7 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void snsFeedWithoutCluster() throws Exception {
         ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertFailed(response);
     }
 
@@ -145,14 +146,11 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
     public void snsDeletedFeed() throws Exception {
         submitFirstClusterScheduleFirstFeed();
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
-        Assert.assertEquals(
-            Util.parseResponse(prism.getFeedHelper()
-                .delete(URLS.DELETE_URL, bundles[0].getDataSets().get(0)))
-                .getStatusCode(), 200);
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().delete(URLS.DELETE_URL,
+            feed)).getStatusCode(), 200);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.KILLED);
-
         ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
     }
@@ -168,14 +166,10 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
         submitFirstClusterScheduleFirstFeed();
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
         Assert.assertEquals(Util.parseResponse(
-                prism.getFeedHelper()
-                    .suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)))
-                .getStatusCode(),
-            200);
+            prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed)).getStatusCode(), 200);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0));
-
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9774414a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
index e0571b5..f20877f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
@@ -98,6 +98,10 @@ public class FeedRetentionTest extends BaseTestClass {
         String inputData = inputPath + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
         String outputPathTemplate = baseHDFSDir +
             "/testOutput/op%d/ivoryRetention0%d/%s/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+        String cluster1Def = bundles[0].getClusters().get(0);
+        String cluster2Def = bundles[1].getClusters().get(0);
+        String cluster1Name = Util.readEntityName(cluster1Def);
+        String cluster2Name = Util.readEntityName(cluster2Def);
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
             TimeUtil.getTimeWrtSystemTime(-5), TimeUtil.getTimeWrtSystemTime(10), 1);
@@ -106,11 +110,10 @@ public class FeedRetentionTest extends BaseTestClass {
         HadoopUtil.flattenAndPutDataInFolder(cluster2FS, OSUtil.RESOURCES + "thriftRRMar0602.gz",
             inputPath, dataDates);
 
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
+        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster1Def);
+        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster2Def);
 
         String feedOutput01 = bundles[0].getFeed("FETL-RequestRC");
-
         feedOutput01 = InstanceUtil.setFeedCluster(feedOutput01,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-01-01T00:00Z"),
             XmlUtil.createRtention("days(10000)", ActionType.DELETE), null,
@@ -119,8 +122,7 @@ public class FeedRetentionTest extends BaseTestClass {
         feedOutput01 = InstanceUtil.setFeedCluster(feedOutput01,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:10Z"),
             XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}",
+            cluster1Name, ClusterType.SOURCE, "${cluster.colo}",
             String.format(outputPathTemplate, 1, 1, "data"),
             String.format(outputPathTemplate, 1, 1, "stats"),
             String.format(outputPathTemplate, 1, 1, "meta"),
@@ -129,8 +131,7 @@ public class FeedRetentionTest extends BaseTestClass {
         feedOutput01 = InstanceUtil.setFeedCluster(feedOutput01,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:25Z"),
             XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}",
+            cluster2Name, ClusterType.SOURCE,"${cluster.colo}",
             String.format(outputPathTemplate, 1, 2, "data"),
             String.format(outputPathTemplate, 1, 2, "stats"),
             String.format(outputPathTemplate, 1, 2, "meta"),
@@ -149,8 +150,7 @@ public class FeedRetentionTest extends BaseTestClass {
         feedOutput02 = InstanceUtil.setFeedCluster(feedOutput02,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:10Z"),
             XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}",
+            cluster1Name, ClusterType.SOURCE, "${cluster.colo}",
             String.format(outputPathTemplate, 2, 1, "data"),
             String.format(outputPathTemplate, 2, 1, "stats"),
             String.format(outputPathTemplate, 2, 1, "meta"),
@@ -159,8 +159,7 @@ public class FeedRetentionTest extends BaseTestClass {
         feedOutput02 = InstanceUtil.setFeedCluster(feedOutput02,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:25Z"),
             XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}",
+            cluster2Name, ClusterType.SOURCE, "${cluster.colo}",
             String.format(outputPathTemplate, 2, 2, "data"),
             String.format(outputPathTemplate, 2, 2, "stats"),
             String.format(outputPathTemplate, 2, 2, "meta"),
@@ -171,8 +170,7 @@ public class FeedRetentionTest extends BaseTestClass {
             prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOutput02));
 
         String feedInput = bundles[0].getFeed("FETL2-RRLog");
-        feedInput = InstanceUtil
-            .setFeedCluster(feedInput,
+        feedInput = InstanceUtil.setFeedCluster(feedInput,
                 XmlUtil.createValidity("2010-10-01T12:00Z", "2099-01-01T00:00Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE), null,
                 ClusterType.SOURCE, null);
@@ -180,14 +178,12 @@ public class FeedRetentionTest extends BaseTestClass {
         feedInput = InstanceUtil.setFeedCluster(feedInput,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:10Z"),
             XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}", inputData);
+            cluster1Name, ClusterType.SOURCE, "${cluster.colo}", inputData);
 
         feedInput = InstanceUtil.setFeedCluster(feedInput,
             XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:25Z"),
             XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}", inputData);
+            cluster2Name, ClusterType.SOURCE, "${cluster.colo}", inputData);
 
         AssertUtil.assertSucceeded(
             prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedInput));
@@ -196,22 +192,17 @@ public class FeedRetentionTest extends BaseTestClass {
         process = InstanceUtil.setProcessCluster(process, null,
             XmlUtil.createProcessValidity("2012-10-01T12:00Z", "2012-10-01T12:10Z"));
 
-        process = InstanceUtil.setProcessCluster(process,
-            Util.readEntityName(bundles[0].getClusters().get(0)),
+        process = InstanceUtil.setProcessCluster(process, cluster1Name,
             XmlUtil.createProcessValidity(TimeUtil.getTimeWrtSystemTime(-2),
                 TimeUtil.getTimeWrtSystemTime(5)));
-        process = InstanceUtil.setProcessCluster(process,
-            Util.readEntityName(bundles[1].getClusters().get(0)),
+        process = InstanceUtil.setProcessCluster(process, cluster2Name,
             XmlUtil.createProcessValidity(TimeUtil.getTimeWrtSystemTime(-2),
                 TimeUtil.getTimeWrtSystemTime(5)));
 
         logger.info("process: " + Util.prettyPrintXml(process));
-
         AssertUtil.assertSucceeded(
             prism.getProcessHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process));
-
         AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOutput01));
         AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOutput02));
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9774414a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
index 4b8fe0a..01c00a9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
@@ -81,28 +81,24 @@ public class OptionalInputTest extends BaseTestClass {
      */
     @Test(enabled = true, groups = {"singleCluster"})
     public void optionalTest_1optional_1compulsary() throws Exception {
-        bundles[0].generateRequiredBundle(1, 2, 1, inputPath, 1, "2010-01-02T01:00Z",
-            "2010-01-02T01:12Z");
+        bundles[0].generateRequiredBundle(1, 2, 1, inputPath, 1,
+                "2010-01-02T01:00Z", "2010-01-02T01:12Z");
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
         bundles[0].setProcessConcurrency(2);
-        logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        logger.info(Util.prettyPrintXml(process));
 
         bundles[0].submitAndScheduleBundle(prism, false);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide("2010-01-02T00:50Z",
             "2010-01-02T01:10Z", 5);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input1/", dataDates);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, Util.getProcessName(process),
                 2, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
     }
 
@@ -115,37 +111,30 @@ public class OptionalInputTest extends BaseTestClass {
      */
     @Test(enabled = true, groups = {"singleCluster"})
     public void optionalTest_1optional_2compulsary() throws Exception {
-        bundles[0].generateRequiredBundle(1, 3, 1, inputPath, 1, "2010-01-02T01:00Z",
-            "2010-01-02T01:12Z");
-
+        bundles[0].generateRequiredBundle(1, 3, 1, inputPath, 1,
+                "2010-01-02T01:00Z", "2010-01-02T01:12Z");
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
         bundles[0].setProcessConcurrency(2);
+        String processName = Util.readEntityName(bundles[0].getProcessData());
         logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
-
         bundles[0].submitAndScheduleBundle(prism, false);
 
         logger.info("instanceShouldStillBeInWaitingState");
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.WAITING, EntityType.PROCESS);
 
-        List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide("2010-01-02T00:50Z",
-            "2010-01-02T01:10Z", 5);
+        List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
+                "2010-01-02T00:50Z", "2010-01-02T01:10Z", 5);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input1/", dataDates);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input2/", dataDates);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
     }
 
@@ -160,32 +149,24 @@ public class OptionalInputTest extends BaseTestClass {
     public void optionalTest_2optional_1compulsary() throws Exception {
         bundles[0].generateRequiredBundle(1, 3, 2, inputPath, 1, "2010-01-02T01:00Z",
             "2010-01-02T01:12Z");
-
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
         bundles[0].setProcessConcurrency(2);
+        String processName = Util.readEntityName(bundles[0].getProcessData());
         logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
 
         bundles[0].submitAndScheduleBundle(prism, false);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.WAITING, EntityType.PROCESS);
-
-        List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide("2010-01-02T00:50Z",
-            "2010-01-02T01:10Z", 5);
+        List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
+                "2010-01-02T00:50Z", "2010-01-02T01:10Z", 5);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input2/", dataDates);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
     }
 
@@ -201,16 +182,15 @@ public class OptionalInputTest extends BaseTestClass {
         String startTime = TimeUtil.getTimeWrtSystemTime(-4);
         String endTime = TimeUtil.getTimeWrtSystemTime(10);
         bundles[0].generateRequiredBundle(1, 2, 1, inputPath, 1, startTime, endTime);
-
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
         bundles[0].setProcessConcurrency(2);
-        logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        logger.info(Util.prettyPrintXml(process));
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
             TimeUtil.addMinsToTime(startTime, -10), endTime, 5);
@@ -220,12 +200,8 @@ public class OptionalInputTest extends BaseTestClass {
         for (String date : dataDates) {
             HadoopUtil.recreateDir(clusterFS, inputPath + "/input0/" + date);
         }
-
         bundles[0].submitFeedsScheduleProcess(prism);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, Util.getProcessName(process),
                 2, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
     }
 
@@ -237,24 +213,19 @@ public class OptionalInputTest extends BaseTestClass {
      */
     @Test(enabled = true, groups = {"singleCluster"})
     public void optionalTest_allInputOptional() throws Exception {
-        bundles[0].generateRequiredBundle(1, 2, 2, inputPath, 1, "2010-01-02T01:00Z",
-            "2010-01-02T01:12Z");
-
+        bundles[0].generateRequiredBundle(1, 2, 2, inputPath, 1,
+                "2010-01-02T01:00Z", "2010-01-02T01:12Z");
         bundles[0].setProcessInputNames("inputData");
-
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
-        logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        logger.info(Util.prettyPrintXml(process));
 
         bundles[0].submitAndScheduleBundle(prism, false);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, Util.getProcessName(process),
                 2, CoordinatorAction.Status.KILLED, EntityType.PROCESS);
     }
 
@@ -271,55 +242,43 @@ public class OptionalInputTest extends BaseTestClass {
         String startTime = TimeUtil.getTimeWrtSystemTime(-4);
         String endTime = TimeUtil.getTimeWrtSystemTime(30);
         bundles[0].generateRequiredBundle(1, 2, 1, inputPath, 1, startTime, endTime);
-
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
         bundles[0].setProcessConcurrency(2);
-        logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        String processName = Util.getProcessName(process);
+        logger.info(Util.prettyPrintXml(process));
 
         bundles[0].submitAndScheduleBundle(prism, true);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.WAITING, EntityType.PROCESS);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
             TimeUtil.addMinsToTime(startTime, -10), endTime, 5);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input1/", dataDates);
 
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 1, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
 
-        final ProcessMerlin processMerlin = new ProcessMerlin(bundles[0].getProcessData());
+        final ProcessMerlin processMerlin = new ProcessMerlin(process);
         processMerlin.setProcessFeeds(bundles[0].getDataSets(), 2, 0, 1);
         bundles[0].setProcessData(processMerlin.toString());
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
-        logger.info("modified process:" + Util.prettyPrintXml(bundles[0].getProcessData()));
+        process = bundles[0].getProcessData();
+        logger.info("modified process:" + Util.prettyPrintXml(process));
 
-        prism.getProcessHelper().update(bundles[0].getProcessData(), bundles[0].getProcessData());
+        prism.getProcessHelper().update(process, process);
 
         //from now on ... it should wait of input0 also
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.WAITING, EntityType.PROCESS);
-
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input0/", dataDates);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
     }
 
@@ -336,52 +295,42 @@ public class OptionalInputTest extends BaseTestClass {
         String startTime = TimeUtil.getTimeWrtSystemTime(-4);
         String endTime = TimeUtil.getTimeWrtSystemTime(30);
         bundles[0].generateRequiredBundle(1, 2, 1, inputPath, 1, startTime, endTime);
-
         for (int i = 0; i < bundles[0].getClusters().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getClusters().get(i)));
-
         for (int i = 0; i < bundles[0].getDataSets().size(); i++)
             logger.info(Util.prettyPrintXml(bundles[0].getDataSets().get(i)));
 
         bundles[0].setProcessInputStartEnd("now(0,-10)", "now(0,0)");
         bundles[0].setProcessConcurrency(4);
-        logger.info(Util.prettyPrintXml(bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        String processName = Util.getProcessName(process);
+        logger.info(Util.prettyPrintXml(process));
 
         bundles[0].submitAndScheduleBundle(prism, true);
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.WAITING, EntityType.PROCESS);
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
             TimeUtil.addMinsToTime(startTime, -10), TimeUtil.addMinsToTime(endTime, 10), 5);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             inputPath + "/input1/", dataDates);
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 1, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
 
-        final ProcessMerlin processMerlin = new ProcessMerlin(bundles[0].getProcessData());
+        final ProcessMerlin processMerlin = new ProcessMerlin(process);
         processMerlin.setProcessFeeds(bundles[0].getDataSets(), 2, 2, 1);
         bundles[0].setProcessData(processMerlin.toString());
+        process = bundles[0].getProcessData();
 
         //delete all input data
         HadoopUtil.deleteDirIfExists(inputPath + "/", clusterFS);
-
         bundles[0].setProcessInputNames("inputData0", "inputData");
+        logger.info("modified process:" + Util.prettyPrintXml(process));
 
-        logger.info("modified process:" + Util.prettyPrintXml(bundles[0].getProcessData()));
+        prism.getProcessHelper().update(process, process);
 
-        prism.getProcessHelper().update(bundles[0].getProcessData(), bundles[0].getProcessData());
-
-        logger.info("modified process:" + Util.prettyPrintXml(bundles[0].getProcessData()));
         //from now on ... it should wait of input0 also
-
-        InstanceUtil
-            .waitTillInstanceReachState(oozieClient,
-                Util.getProcessName(bundles[0].getProcessData()),
+        InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.KILLED, EntityType.PROCESS);
     }
 }


[22/41] git commit: FALCON-660 7 test classes refactored and few of them documented. Contributed by Paul Isaychuk

Posted by ra...@apache.org.
FALCON-660 7 test classes refactored and few of them documented. Contributed by Paul Isaychuk


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

Branch: refs/heads/FALCON-585
Commit: 3678eabab73603a7ca3e45e1a56187cc0aca3619
Parents: 75f06b4
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Wed Sep 3 18:20:38 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Wed Sep 3 18:20:38 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   3 +
 .../falcon/regression/InstanceParamTest.java    |  45 ++--
 .../falcon/regression/InstanceSummaryTest.java  | 171 +++++--------
 .../regression/prism/EntityDryRunTest.java      |  26 +-
 .../regression/prism/PrismFeedSnSTest.java      | 252 ++++++++++---------
 .../prism/PrismProcessScheduleTest.java         | 111 +++-----
 .../regression/prism/PrismProcessSnSTest.java   | 151 ++++++-----
 .../falcon/regression/ui/ProcessUITest.java     |  21 +-
 8 files changed, 361 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 94403c8..b60c23c 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,6 +9,9 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
+   FALCON-660 7 test classes refactored and few of them documented (Paul Isaychuk via
+   Ruslan Ostafiychuk)
+
    FALCON-653 Add falcon regression test for zero input process(Karishma via Samarth Gupta)
    FALCON-655 Skip workflow upload if process won't be submitted (Ruslan Ostafiychuk)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
index d733cfc..edf3428 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
@@ -58,24 +58,21 @@ public class InstanceParamTest extends BaseTestClass {
      */
 
     private String baseTestHDFSDir = baseHDFSDir + "/InstanceParamTest";
-    private String feedInputPath = baseTestHDFSDir
-            +
+    private String feedInputPath = baseTestHDFSDir +
         "/testInputData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private String startTime;
     private String endTime;
-
     private ColoHelper cluster1 = servers.get(0);
-    private OozieClient oC1 = serverOC.get(0);
+    private OozieClient cluster1OC = serverOC.get(0);
     private Bundle processBundle;
     private static final Logger LOGGER = Logger.getLogger(InstanceParamTest.class);
-
+    private String processName;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
         uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-        startTime = TimeUtil.get20roundedTime(TimeUtil
-            .getTimeWrtSystemTime(-20));
+        startTime = TimeUtil.get20roundedTime(TimeUtil.getTimeWrtSystemTime(-20));
         endTime = TimeUtil.getTimeWrtSystemTime(60);
     }
 
@@ -92,7 +89,12 @@ public class InstanceParamTest extends BaseTestClass {
             bundles[i].generateUniqueBundle();
             bundles[i].setProcessWorkflow(aggregateWorkflowDir);
         }
+        processName = processBundle.getProcessName();
     }
+
+    /**
+     * Schedule process. Get params of waiting instance.
+     */
     @Test(timeOut = 1200000, enabled = false)
     public void getParamsValidRequestInstanceWaiting()
         throws URISyntaxException, JAXBException, AuthenticationException, IOException,
@@ -104,12 +106,14 @@ public class InstanceParamTest extends BaseTestClass {
             ClusterType.SOURCE, null, null);
         processBundle.submitFeedsScheduleProcess(prism);
         InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 0);
-        InstancesResult r = prism.getProcessHelper()
-            .getInstanceParams(Util.readEntityName(processBundle.getProcessData()),
-                "?start="+startTime);
+        InstancesResult r = prism.getProcessHelper().getInstanceParams(processName,
+            "?start=" + startTime);
         r.getMessage();
     }
 
+    /**
+     * Schedule process. Wait till instance succeeded. Get its params.
+     */
     @Test(timeOut = 1200000, enabled = true)
     public void getParamsValidRequestInstanceSucceeded()
         throws URISyntaxException, JAXBException, AuthenticationException, IOException,
@@ -121,16 +125,18 @@ public class InstanceParamTest extends BaseTestClass {
             ClusterType.SOURCE, null, null);
         processBundle.submitFeedsScheduleProcess(prism);
         InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 0);
-        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS,
-            processBundle.getProcessName(), 0);
-        InstanceUtil.waitTillInstanceReachState(oC1, processBundle.getProcessName(), 1,
+        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS, processName, 0);
+        InstanceUtil.waitTillInstanceReachState(cluster1OC, processName, 1,
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS, 10);
         InstancesResult r = prism.getProcessHelper()
-            .getInstanceParams(Util.readEntityName(processBundle.getProcessData()),
-                "?start="+startTime);
+            .getInstanceParams(processName, "?start=" + startTime);
         LOGGER.info(r.getMessage());
     }
 
+    /**
+     *  Schedule process. Wait till instance got killed. Get its params.
+     *  TODO: change according to test case
+     */
     @Test(timeOut = 1200000, enabled = false)
     public void getParamsValidRequestInstanceKilled()
         throws URISyntaxException, JAXBException, AuthenticationException, IOException,
@@ -142,15 +148,12 @@ public class InstanceParamTest extends BaseTestClass {
             ClusterType.SOURCE, null, null);
         processBundle.submitFeedsScheduleProcess(prism);
         InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 0);
-        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS,
-            processBundle.getProcessName(), 0);
-        InstanceUtil.waitTillInstanceReachState(oC1, processBundle.getProcessName(), 0,
+        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS, processName, 0);
+        InstanceUtil.waitTillInstanceReachState(cluster1OC, processName, 0,
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
         InstancesResult r = prism.getProcessHelper()
-            .getInstanceParams(Util.readEntityName(processBundle.getProcessData()),
-                "?start="+startTime);
+            .getInstanceParams(processName, "?start=" + startTime);
         r.getMessage();
-
     }
 
     @AfterMethod(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index 154485f..636da2c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@ -50,43 +50,34 @@ import java.net.URISyntaxException;
 import java.text.ParseException;
 import java.util.List;
 
-/*
-this test currently provide minimum verification. More detailed test should
- be added
+/** This test currently provide minimum verification. More detailed test should be added:
+    1. process : test summary single cluster few instance some future some past
+    2. process : test multiple cluster, full past on one cluster,  full future on one cluster,
+    half future / past on third one
+    3. feed : same as test 1 for feed
+    4. feed : same as test 2 for feed
  */
 @Test(groups = "embedded")
 public class InstanceSummaryTest extends BaseTestClass {
 
-    //1. process : test summary single cluster few instance some future some past
-    //2. process : test multiple cluster, full past on one cluster,
-    // full future on one cluster, half future / past on third one
-
-    // 3. feed : same as test 1 for feed
-    // 4. feed : same as test 2 for feed
-
-
     String baseTestHDFSDir = baseHDFSDir + "/InstanceSummaryTest";
     String feedInputPath = baseTestHDFSDir +
         "/testInputData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     String startTime;
     String endTime;
-
     ColoHelper cluster3 = servers.get(2);
-
     Bundle processBundle;
     private static final Logger logger = Logger.getLogger(InstanceSummaryTest.class);
+    String processName;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
         uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-        startTime = TimeUtil.get20roundedTime(TimeUtil
-            .getTimeWrtSystemTime
-                (-20));
+        startTime = TimeUtil.get20roundedTime(TimeUtil.getTimeWrtSystemTime(-20));
         endTime = TimeUtil.getTimeWrtSystemTime(60);
         String startTimeData = TimeUtil.addMinsToTime(startTime, -100);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startTimeData, endTime, 20);
-
         for (FileSystem fs : serverFS) {
             HadoopUtil.deleteDirIfExists(Util.getPathPrefix(feedInputPath), fs);
             HadoopUtil.flattenAndPutDataInFolder(fs, OSUtil.NORMAL_INPUT,
@@ -108,100 +99,88 @@ public class InstanceSummaryTest extends BaseTestClass {
             bundles[i].generateUniqueBundle();
             bundles[i].setProcessWorkflow(aggregateWorkflowDir);
         }
+        processName = Util.readEntityName(processBundle.getProcessData());
     }
 
+    /**
+     *  Schedule single-cluster process. Get its instances summary.
+     *  TODO: should be complete
+     */
     @Test(enabled = true, timeOut = 1200000)
     public void testSummarySingleClusterProcess()
         throws URISyntaxException, JAXBException, IOException, ParseException,
         OozieClientException, AuthenticationException {
         processBundle.setProcessValidity(startTime, endTime);
         processBundle.submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster3,
-            processBundle.getProcessData(), 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster3, processBundle.getProcessData(), 0);
 
         // start only at start time
         InstancesSummaryResult r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime);
-
-        InstanceUtil.waitTillInstanceReachState(serverOC.get(2),
-            Util.readEntityName(processBundle.getProcessData()), 2,
+            .getInstanceSummary(processName, "?start=" + startTime);
+        InstanceUtil.waitTillInstanceReachState(serverOC.get(2), processName, 2,
             Status.SUCCEEDED, EntityType.PROCESS);
 
-
         //AssertUtil.assertSucceeded(r);
 
         //start only before process start
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
+        r = prism.getProcessHelper().getInstanceSummary(processName,
                 "?start=" + TimeUtil.addMinsToTime(startTime, -100));
         //AssertUtil.assertFailed(r,"response should have failed");
 
         //start only after process end
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
+        r = prism.getProcessHelper().getInstanceSummary(processName,
                 "?start=" + TimeUtil.addMinsToTime(startTime, 120));
 
 
         //start only at mid specific instance
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    +10));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+                "?start=" + TimeUtil.addMinsToTime(startTime, 10));
 
         //start only in between 2 instance
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    7));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+                "?start=" + TimeUtil.addMinsToTime(startTime, 7));
 
         //start and end at start and end
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + startTime + "&end=" + endTime);
 
         //start in between and end at end
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    14) + "&end=" + endTime);
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + TimeUtil.addMinsToTime(startTime, 14) + "&end=" + endTime);
 
         //start at start and end between
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime,
-                    -20));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime, -20));
 
         // start and end in between
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    20) + "&end=" + TimeUtil.addMinsToTime(endTime, -13));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+                "?start=" + TimeUtil.addMinsToTime(startTime, 20)
+                    + "&end=" + TimeUtil.addMinsToTime(endTime, -13));
 
         //start before start with end in between
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    -100) + "&end=" + TimeUtil.addMinsToTime(endTime, -37));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+                "?start=" + TimeUtil.addMinsToTime(startTime, -100)
+                    + "&end=" + TimeUtil.addMinsToTime(endTime, -37));
 
         //start in between and end after end
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    60) + "&end=" + TimeUtil.addMinsToTime(endTime, 100));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+                "?start=" + TimeUtil.addMinsToTime(startTime, 60)
+                    + "&end=" + TimeUtil.addMinsToTime(endTime, 100));
 
         // both start end out od range
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + TimeUtil.addMinsToTime(startTime,
-                    -100) + "&end=" + TimeUtil.addMinsToTime(endTime, 100));
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + TimeUtil.addMinsToTime(startTime,-100)
+                + "&end=" + TimeUtil.addMinsToTime(endTime, 100));
 
         // end only
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
+        r = prism.getProcessHelper().getInstanceSummary(processName,
                 "?end=" + TimeUtil.addMinsToTime(endTime, -30));
     }
 
+    /**
+     * Adjust multi-cluster process. Submit and schedule it. Get its instances summary.
+     * TODO: should be complete
+     */
     @Test(enabled = true, timeOut = 1200000)
     public void testSummaryMultiClusterProcess() throws JAXBException,
         ParseException, IOException, URISyntaxException, AuthenticationException {
@@ -212,39 +191,31 @@ public class InstanceSummaryTest extends BaseTestClass {
             ClusterType.SOURCE, null, null);
         processBundle.submitFeedsScheduleProcess(prism);
         InstancesSummaryResult r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime);
-
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
+            .getInstanceSummary(processName, "?start=" + startTime);
 
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+             "?start=" + startTime + "&end=" + endTime);
 
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+             "?start=" + startTime + "&end=" + endTime);
 
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + startTime + "&end=" + endTime);
 
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + startTime + "&end=" + endTime);
 
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + startTime + "&end=" + endTime);
 
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
-
-
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
-
-
-        r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime + "&end=" + endTime);
+        r = prism.getProcessHelper().getInstanceSummary(processName,
+            "?start=" + startTime + "&end=" + endTime);
     }
 
+    /**
+     *  Adjust multi-cluster feed. Submit and schedule it. Get its instances summary.
+     *  TODO: should be complete
+     */
     @Test(enabled = true, timeOut = 1200000)
     public void testSummaryMultiClusterFeed() throws JAXBException, ParseException, IOException,
         URISyntaxException, OozieClientException, AuthenticationException {
@@ -253,7 +224,6 @@ public class InstanceSummaryTest extends BaseTestClass {
         String feed = bundles[0].getDataSets().get(0);
 
         //cluster_1 is target, cluster_2 is source and cluster_3 is neutral
-
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2010-01-01T00:00Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE), null,
@@ -268,8 +238,7 @@ public class InstanceSummaryTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTime, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(100000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.TARGET,
-                null,
-                feedInputPath);
+                null, feedInputPath);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTime, "2099-01-01T00:00Z"),
@@ -287,21 +256,15 @@ public class InstanceSummaryTest extends BaseTestClass {
         feedInputPath, 1);*/
 
         //submit and schedule feed
-        prism.getFeedHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
 
         InstancesSummaryResult r = prism.getFeedHelper()
-            .getInstanceSummary(Util.readEntityName(feed),
-                "?start=" + startTime);
-
-        r = prism.getFeedHelper()
-            .getInstanceSummary(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime,
-                    -20));
+            .getInstanceSummary(Util.readEntityName(feed), "?start=" + startTime);
 
+        r = prism.getFeedHelper().getInstanceSummary(Util.readEntityName(feed),
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime, -20));
     }
 
-
     @AfterMethod(alwaysRun = true)
     public void tearDown() throws IOException {
         processBundle.deleteBundle(prism);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
index 5000746..0b06823 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
@@ -52,9 +52,9 @@ public class EntityDryRunTest extends BaseTestClass {
     private FileSystem clusterFS = serverFS.get(0);
     private String baseTestHDFSDir = baseHDFSDir + "/EntityDryRunTest";
     private String feedInputPath = baseTestHDFSDir +
-            "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+        "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private String feedOutputPath =
-            baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private static final Logger LOGGER = Logger.getLogger(EntityDryRunTest.class);
 
@@ -85,7 +85,6 @@ public class EntityDryRunTest extends BaseTestClass {
     }
 
     /**
-     *
      * tries to submit process with invalid el exp
      */
     @Test(groups = {"singleCluster"})
@@ -93,12 +92,11 @@ public class EntityDryRunTest extends BaseTestClass {
         bundles[0].setProcessProperty("EntityDryRunTestProp", "${coord:someEL(1)");
         bundles[0].submitProcess(true);
         ServiceResponse response = prism.getProcessHelper()
-                .schedule(Util.URLS.SCHEDULE_URL, bundles[0].getProcessData());
+            .schedule(Util.URLS.SCHEDULE_URL, bundles[0].getProcessData());
         validate(response);
     }
 
     /**
-     *
      * tries to update process with invalid EL exp
      */
     @Test(groups = {"singleCluster"})
@@ -108,17 +106,15 @@ public class EntityDryRunTest extends BaseTestClass {
         bundles[0].submitAndScheduleProcess();
         bundles[0].setProcessProperty("EntityDryRunTestProp", "${coord:someEL(1)");
         ServiceResponse response = prism.getProcessHelper().update(bundles[0].getProcessData(),
-                bundles[0].getProcessData(), TimeUtil.getTimeWrtSystemTime(5), null);
+            bundles[0].getProcessData(), TimeUtil.getTimeWrtSystemTime(5), null);
         validate(response);
         Assert.assertEquals(
             OozieUtil.getNumberOfBundle(cluster, EntityType.PROCESS, bundles[0].getProcessName()),
-            1,
-            "more than one bundle found after failed update request");
+            1, "more than one bundle found after failed update request");
     }
 
     /**
      * tries to submit feed with invalied EL exp
-     *
      */
     @Test(groups = {"singleCluster"})
     public void testDryRunFailureScheduleFeed() throws Exception {
@@ -131,14 +127,14 @@ public class EntityDryRunTest extends BaseTestClass {
     }
 
     /**
-     *
      * tries to update feed with invalid el exp
      */
     @Test(groups = {"singleCluster"})
     public void testDryRunFailureUpdateFeed() throws Exception {
         bundles[0].submitClusters(prism);
         String feed = bundles[0].getInputFeedFromBundle();
-        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response =
+            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(response);
         feed = Util.setFeedProperty(feed, "EntityDryRunTestProp", "${coord:someEL(1)");
         response = prism.getFeedHelper().update(feed, feed);
@@ -150,11 +146,9 @@ public class EntityDryRunTest extends BaseTestClass {
 
     private void validate(ServiceResponse response) throws JAXBException {
         AssertUtil.assertFailed(response);
-        Assert.assertTrue(response.getMessage()
-            .contains("org.apache.falcon.FalconException: AUTHENTICATION : E1004 :" +
-                " E1004: Expression language evaluation error, Unable to evaluate :${coord:someEL" +
-                "(1)"),
-            "Correct response was not present in process / feed schedule");
+        Assert.assertTrue(response.getMessage().contains("org.apache.falcon.FalconException: " +
+            "AUTHENTICATION : E1004 : Expression language evaluation error, Unable to evaluate " +
+            ":${coord:someEL(1)"), "Correct response was not present in process / feed schedule");
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
index b7da224..15a2c3c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
@@ -56,6 +56,7 @@ public class PrismFeedSnSTest extends BaseTestClass {
     private boolean restartRequired;
     String aggregateWorkflowDir = baseHDFSDir + "/PrismFeedSnSTest/aggregator";
     private static final Logger logger = Logger.getLogger(PrismFeedSnSTest.class);
+    String feed1, feed2;
 
     @BeforeClass(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
@@ -72,6 +73,8 @@ public class PrismFeedSnSTest extends BaseTestClass {
             bundles[i].generateUniqueBundle();
             bundles[i].setProcessWorkflow(aggregateWorkflowDir);
         }
+        feed1 = bundles[0].getDataSets().get(0);
+        feed2 = bundles[1].getDataSets().get(0);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -82,22 +85,27 @@ public class PrismFeedSnSTest extends BaseTestClass {
         removeBundles();
     }
 
-
+    /**
+     *  Submit and schedule feed1 on cluster1 and check that only this feed is running there.
+     *  Perform the same for feed2 on cluster2.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        bundles[1].submitAndScheduleFeed();
 
-        //now check if they have been scheduled correctly or not
+        bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
-
-        //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
 
+    /**
+     *  Submit and schedule feed1 on cluster1 and feed2 on cluster2. Check that they are running
+     *  on matching clusters only. Submit and schedule them once more. Check that new bundles
+     *  were not created and feed still running on matching clusters.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testSnSAlreadyScheduledFeedOnBothColos() throws Exception {
         //schedule both bundles
@@ -112,102 +120,105 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster2OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
-
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         //ensure only one bundle is there
         Assert.assertEquals(OozieUtil.getBundles(cluster1OC,
-            Util.readEntityName(bundles[0].getDataSets().get(0)), EntityType.FEED).size(), 1);
+            Util.readEntityName(feed1), EntityType.FEED).size(), 1);
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        //ensure only one bundle is there
         Assert.assertEquals(OozieUtil.getBundles(cluster2OC,
-            Util.readEntityName(bundles[1].getDataSets().get(0)), EntityType.FEED).size(), 1);
+            Util.readEntityName(feed2), EntityType.FEED).size(), 1);
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
-
+    /**
+     * Submit and schedule feed1 on cluster1, feed2 on cluster2. Suspend feed1 and check their
+     * statuses. Submit and schedule feed1 again. Check that statuses hasn't been changed and new
+     * bundle hasn't been created. Resume feed1. Repeat the same for feed2.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testSnSSuspendedFeedOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
-        //now check if they have been scheduled correctly or not
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         Assert.assertEquals(OozieUtil.getBundles(cluster1OC,
-            Util.readEntityName(bundles[0].getDataSets().get(0)), EntityType.FEED).size(), 1);
+            Util.readEntityName(feed1), EntityType.FEED).size(), 1);
 
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-            .resume(URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().resume(URLS.RESUME_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         Assert.assertEquals(OozieUtil.getBundles(cluster2OC,
-            Util.readEntityName(bundles[1].getDataSets().get(0)), EntityType.FEED).size(), 1);
-        AssertUtil.assertSucceeded(cluster2.getFeedHelper()
-            .resume(URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+            Util.readEntityName(feed2), EntityType.FEED).size(), 1);
+        AssertUtil.assertSucceeded(cluster2.getFeedHelper().resume(URLS.RESUME_URL, feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
-
-
     }
 
+    /**
+     *  Submit and schedule both feeds. Delete them and submit and schedule again. Check that
+     *  action succeeded.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testSnSDeletedFeedOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
 
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
     }
 
+    /**
+     *  Attempt to submit and schedule non-registered feed should fail.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testScheduleNonExistentFeedOnBothColos() throws Exception {
         AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
     }
 
+    /**
+     *  Shut down server on cluster1. Submit and schedule feed on cluster2. Check that only
+     *  mentioned feed is running there.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOn1ColoWhileOtherColoIsDown() throws Exception {
         restartRequired = true;
-        for (String cluster : bundles[1].getClusters()) {
-            AssertUtil
-                .assertSucceeded(prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster));
-        }
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL,
+            bundles[1].getClusters().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
-
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
 
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -215,24 +226,29 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
 
-
+    /**
+     *  Attempt to submit and schedule feed on cluster which is down should fail and this feed
+     *  shouldn't run on another cluster.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOn1ColoWhileThatColoIsDown() throws Exception {
         restartRequired = true;
         bundles[0].submitFeed();
-
         Util.shutDownService(cluster1.getFeedHelper());
-
         AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
 
+    /**
+     *  Submit and schedule and then suspend feed1 on cluster1. Submit and schedule feed2 on
+     *  cluster2 and check that this actions don't affect each other.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasSuspendedFeed() throws Exception {
         bundles[0].submitAndScheduleFeed();
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+            .suspend(URLS.SUSPEND_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -241,11 +257,15 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
+    /**
+     *  Submit and schedule and then delete feed1 on cluster1. Submit and schedule feed2 on
+     *  cluster2 and check that this actions don't affect each other.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasKilledFeed() throws Exception {
         bundles[0].submitAndScheduleFeed();
         AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+            prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -254,108 +274,110 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
+    /**
+     * Submit and schedule feed1 on cluster1 and check that it failed. Repeat for feed2.
+     *  TODO: should be reviewed
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOnBothColosUsingColoHelper() throws Exception {
         //schedule both bundles
         bundles[0].submitFeed();
         APIResult result = Util.parseResponse((cluster1.getFeedHelper()
-            .submitEntity(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0))));
+            .submitEntity(URLS.SUBMIT_AND_SCHEDULE_URL, feed1)));
         Assert.assertEquals(result.getStatusCode(), 404);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
         bundles[1].submitFeed();
         result = Util.parseResponse(cluster2.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
         Assert.assertEquals(result.getStatusCode(), 404);
-
         AssertUtil.checkNotStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
-
+    /**
+     *  Submit and schedule both feeds. Suspend feed1 and submit and schedule it once more. Check
+     *  that status of feed1 is still suspended. Resume it. Suspend feed2 but submit and schedule
+     *  feed1 again. Check that it didn't affect feed2 and it is still suspended.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testSnSSuspendedFeedOnBothColosUsingColoHelper() throws Exception {
-
         //schedule both bundles
         bundles[0].submitFeed();
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         bundles[1].submitFeed();
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
 
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().suspend(URLS.SUSPEND_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
         //now check if they have been scheduled correctly or not
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(
-            cluster1.getFeedHelper().resume(URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().resume(URLS.RESUME_URL, feed1));
 
-        AssertUtil.assertSucceeded(cluster2.getFeedHelper().suspend(URLS.SUSPEND_URL,
-            bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(cluster2.getFeedHelper().suspend(URLS.SUSPEND_URL, feed2));
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
     }
 
-
+    /**
+     *  Submit and schedule both feeds and then delete them. Submit and schedule feeds again.
+     *  Check that action succeeded and feeds are running.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testScheduleDeletedFeedOnBothColosUsingColoHelper() throws Exception {
-
         //schedule both bundles
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
-
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper()
-                .getStatus(URLS.STATUS_URL, bundles[0].getDataSets().get(0))).getMessage(),
-            cluster1.getClusterHelper().getColoName() + "/RUNNING");
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
 
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper()
-                .getStatus(URLS.STATUS_URL, bundles[1].getDataSets().get(0))).getMessage(),
-            cluster2.getClusterHelper().getColoName() + "/RUNNING");
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(URLS.STATUS_URL,
+            feed1)).getMessage(), cluster1.getClusterHelper().getColoName() + "/RUNNING");
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(URLS.STATUS_URL,
+            feed2)).getMessage(), cluster2.getClusterHelper().getColoName() + "/RUNNING");
     }
 
-
+    /**
+     *  Attempt to submit and schedule non-registered feeds should fail.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testSNSNonExistentFeedOnBothColosUsingColoHelper() throws Exception {
-
         Assert.assertEquals(Util.parseResponse(cluster1.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)))
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1))
             .getStatusCode(), 404);
         Assert.assertEquals(Util.parseResponse(cluster2.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)))
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2))
             .getStatusCode(), 404);
     }
 
+    /**
+     *  Shut down server on cluster1. Submit and schedule feed on cluster2. Check that only that
+     *  feed is running on cluster2.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOn1ColoWhileOtherColoIsDownUsingColoHelper() throws Exception {
         restartRequired = true;
-        for (String cluster : bundles[1].getClusters()) {
-            AssertUtil
-                .assertSucceeded(prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster));
-        }
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL,
+            bundles[1].getClusters().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
-
         AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
 
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -363,21 +385,24 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
 
-
+    /**
+     *  Set both clusters as feed clusters. Shut down one of them. Submit and schedule feed.
+     *  Check that action is partially successful.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOn1ColoWhileThatColoIsDownUsingColoHelper() throws Exception {
         restartRequired = true;
+        String clust1 = bundles[0].getClusters().get(0);
+        String clust2 = bundles[1].getClusters().get(0);
 
         bundles[0].setCLusterColo(cluster1.getClusterHelper().getColoName());
-        logger.info("cluster bundles[0]: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
-
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        logger.info("cluster bundles[0]: " + Util.prettyPrintXml(clust1));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, clust1);
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         bundles[1].setCLusterColo(cluster2.getClusterHelper().getColoName());
-        logger.info("cluster bundles[1]: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
+        logger.info("cluster bundles[1]: " + Util.prettyPrintXml(clust2));
+        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, clust2);
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         String startTimeUA1 = "2012-10-01T12:00Z";
@@ -392,53 +417,52 @@ public class PrismFeedSnSTest extends BaseTestClass {
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2099-10-01T12:10Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
-                Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-                "${cluster.colo}",
+                Util.readEntityName(clust1), ClusterType.SOURCE, "${cluster.colo}",
                 baseHDFSDir + "/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
-
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
-                Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                baseHDFSDir +
+                Util.readEntityName(clust2), ClusterType.TARGET, null, baseHDFSDir +
                     "/clusterPath/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
-
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
-        AssertUtil.assertPartial(response);
-        response = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
-        AssertUtil.assertPartial(response);
+        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        AssertUtil.assertPartial(r);
+        r = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        AssertUtil.assertPartial(r);
         Util.startService(cluster1.getFeedHelper());
-        prism.getClusterHelper().delete(URLS.DELETE_URL, bundles[0].getClusters().get(0));
-        prism.getClusterHelper().delete(URLS.DELETE_URL, bundles[1].getClusters().get(0));
-
+        prism.getClusterHelper().delete(URLS.DELETE_URL, clust1);
+        prism.getClusterHelper().delete(URLS.DELETE_URL, clust2);
     }
 
-
+    /**
+     *   Submit and schedule feed1 and suspend it. Submit and schedule feed2 on another cluster
+     *   and check that only feed2 is running on cluster2.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasSuspendedFeedUsingColoHelper()
         throws Exception {
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(
-            cluster1.getFeedHelper().suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().suspend(URLS.SUSPEND_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
 
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
-        AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
+        AssertUtil.checkNotStatus(cluster2OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
-        AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
+        AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
-
+    /**
+     *  Submit and schedule and delete feed1. Submit and schedule feed2 and check that this
+     *  action didn't affect feed1 and it is still killed.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasKilledFeedUsingColoHelper() throws Exception {
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
index 10df6c2..5590c54 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
@@ -56,6 +56,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
     String aggregateWorkflowDir = baseHDFSDir + "/PrismProcessScheduleTest/aggregator";
     String workflowForNoIpOp = baseHDFSDir + "/PrismProcessScheduleTest/noinop";
     private static final Logger logger = Logger.getLogger(PrismProcessScheduleTest.class);
+    String process1;
+    String process2;
 
     @BeforeClass(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
@@ -72,6 +74,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
             bundles[i].generateUniqueBundle();
             bundles[i].setProcessWorkflow(aggregateWorkflowDir);
         }
+        process1 = bundles[0].getProcessData();
+        process2 = bundles[1].getProcessData();
     }
 
     @AfterMethod(alwaysRun = true)
@@ -99,7 +103,6 @@ public class PrismProcessScheduleTest extends BaseTestClass {
 
         //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
-
     }
 
     /**
@@ -123,9 +126,9 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+            .schedule(URLS.SCHEDULE_URL, process1));
         AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+            .schedule(URLS.SCHEDULE_URL, process2));
 
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
@@ -148,33 +151,27 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcess();
 
         //suspend process on colo-1
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().suspend(URLS.SUSPEND_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //now check if it has been scheduled correctly or not
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .resume(URLS.RESUME_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(URLS.RESUME_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         //suspend process on colo-2
-        AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(URLS.SUSPEND_URL, process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         //now check if it has been scheduled correctly or not
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .resume(URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(URLS.RESUME_URL, process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-  }
+    }
 
     /**
      * Schedule two processes on different colos. Delete both of them. Try to schedule them once
@@ -188,21 +185,16 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         bundles[0].submitAndScheduleProcess();
         bundles[1].submitAndScheduleProcess();
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process2));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
 
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
-
+        AssertUtil.assertFailed(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().schedule(URLS.SCHEDULE_URL, process2));
     }
 
     /**
@@ -212,11 +204,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testScheduleNonExistentProcessOnBothColos() throws Exception {
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
-
+        AssertUtil.assertFailed(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().schedule(URLS.SCHEDULE_URL, process2));
     }
 
     /**
@@ -230,11 +219,9 @@ public class PrismProcessScheduleTest extends BaseTestClass {
     public void testProcessScheduleOn1ColoWhileOtherColoIsDown() throws Exception {
         try {
             bundles[1].submitProcess(true);
-
             Util.shutDownService(cluster2.getProcessHelper());
-
             AssertUtil.assertSucceeded(prism.getProcessHelper()
-                .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getProcessData()));
+                .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2));
 
             //now check if they have been scheduled correctly or not
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
@@ -260,11 +247,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
     public void testProcessScheduleOn1ColoWhileThatColoIsDown() throws Exception {
         try {
             bundles[0].submitProcess(true);
-
             Util.shutDownService(cluster2.getProcessHelper());
-
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
             AssertUtil
                 .checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         } catch (Exception e) {
@@ -273,7 +257,6 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         } finally {
             Util.restartService(cluster2.getProcessHelper());
         }
-
     }
 
     /**
@@ -289,7 +272,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         try {
             bundles[0].submitAndScheduleProcess();
             AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-                .suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
+                .suspend(URLS.SUSPEND_URL, process1));
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
 
@@ -301,12 +284,10 @@ public class PrismProcessScheduleTest extends BaseTestClass {
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
             AssertUtil
                 .checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-
         } catch (Exception e) {
             e.printStackTrace();
             throw new TestNGException(e.getMessage());
         }
-
     }
 
     /**
@@ -321,8 +302,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         throws Exception {
         try {
             bundles[0].submitAndScheduleProcess();
-            AssertUtil.assertSucceeded(prism.getProcessHelper()
-                .delete(URLS.DELETE_URL, bundles[0].getProcessData()));
+            AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process1));
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
 
             bundles[1].submitAndScheduleProcess();
@@ -346,47 +326,30 @@ public class PrismProcessScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"prism", "0.2", "embedded"}, enabled = true, timeOut = 1800000)
     public void testRescheduleKilledProcess() throws Exception {
-
-    /*
-    add test data generator pending
-     */
-
+        /* add test data generator pending */
         bundles[0].setProcessValidity(TimeUtil.getTimeWrtSystemTime(-1),
-            TimeUtil.getTimeWrtSystemTime(1));
+               TimeUtil.getTimeWrtSystemTime(1));
         HadoopFileEditor hadoopFileEditor = null;
+        String process = bundles[0].getProcessData();
         try {
-
-            hadoopFileEditor = new HadoopFileEditor(cluster1
-                .getClusterHelper().getHadoopFS());
-
-            hadoopFileEditor.edit(new ProcessMerlin(bundles[0]
-                .getProcessData()).getWorkflow().getPath() + "/workflow.xml",
-                "<value>${outputData}</value>",
-                "<property>\n" +
-                    "                    <name>randomProp</name>\n" +
-                    "                    <value>randomValue</value>\n" +
-                    "                </property>");
-
+            hadoopFileEditor = new HadoopFileEditor(cluster1.getClusterHelper().getHadoopFS());
+            hadoopFileEditor.edit(new ProcessMerlin(process).getWorkflow().getPath() +
+                    "/workflow.xml", "<value>${outputData}</value>",
+                                        "<property>\n" +
+                       "                    <name>randomProp</name>\n" +
+                       "                    <value>randomValue</value>\n" +
+                       "                </property>");
             bundles[0].submitFeedsScheduleProcess(prism);
-
             InstanceUtil.waitForBundleToReachState(cluster1,
-                Util.readEntityName(bundles[0].getProcessData()),
-                org.apache.oozie.client.Job.Status.KILLED);
-
+                Util.readEntityName(process),Job.Status.KILLED);
             String oldBundleID = InstanceUtil.getLatestBundleID(cluster1,
-                Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS);
-
-            prism.getProcessHelper().delete(URLS.DELETE_URL,
-                bundles[0].getProcessData());
+                Util.readEntityName(process), EntityType.PROCESS);
+            prism.getProcessHelper().delete(URLS.DELETE_URL, process);
 
             bundles[0].submitAndScheduleProcess();
-
             OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID,
-                new ArrayList<String>(),
-                bundles[0].getProcessData(), true,
-                false);
+                new ArrayList<String>(), process, true, false);
         } finally {
-
             if (hadoopFileEditor != null) {
                 hadoopFileEditor.restore();
             }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
index 842bc1a..304549d 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
@@ -50,6 +50,8 @@ public class PrismProcessSnSTest extends BaseTestClass {
     OozieClient cluster2OC = serverOC.get(1);
     String aggregateWorkflowDir = baseHDFSDir + "/PrismProcessSnSTest/aggregator";
     private static final Logger logger = Logger.getLogger(PrismProcessSnSTest.class);
+    String process1;
+    String process2;
 
     @BeforeClass(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
@@ -65,6 +67,8 @@ public class PrismProcessSnSTest extends BaseTestClass {
             bundles[i].generateUniqueBundle();
             bundles[i].setProcessWorkflow(aggregateWorkflowDir);
         }
+        process1 = bundles[0].getProcessData();
+        process2 = bundles[1].getProcessData();
     }
 
     @AfterMethod(alwaysRun = true)
@@ -72,6 +76,11 @@ public class PrismProcessSnSTest extends BaseTestClass {
         removeBundles();
     }
 
+    /**
+     * Submit and schedule process1 on cluster1. Check that process2 is not running on cluster1.
+     * Submit and schedule process2 on cluster2. Check that process2 is running and process1 is
+     * not running on cluster2.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testProcessSnSOnBothColos() throws Exception {
         //schedule both bundles
@@ -79,49 +88,49 @@ public class PrismProcessSnSTest extends BaseTestClass {
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         bundles[1].submitAndScheduleProcess();
-
-        //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
+
         //check if there is no criss cross
-        ServiceResponse response =
-            prism.getProcessHelper()
-                .getStatus(URLS.STATUS_URL, bundles[1].getProcessData());
+        ServiceResponse response = prism.getProcessHelper().getStatus(URLS.STATUS_URL, process2);
         logger.info(response.getMessage());
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
-
     }
 
+    /**
+     * Submit process1 on cluster1 and schedule it. Check that process1 runs on cluster1 but not
+     * on cluster2. Submit process2 but schedule process1 once more. Check that process1 is running
+     * on cluster1 but not on cluster2.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testProcessSnSForSubmittedProcessOnBothColos() throws Exception {
         //schedule both bundles
-
         bundles[0].submitProcess(true);
-
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         bundles[1].submitProcess(true);
 
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
-        //now check if they have been scheduled correctly or not
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
-
     }
 
+    /**
+     * Submit process1 on cluster1 and schedule it. Check that only process1 runs on cluster1.
+     * Submit process2 and check that it isn't running on cluster1. Submit and schedule process1
+     * once more and check that it is still running on cluster1 but process2 isn't running on
+     * cluster2.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testProcessSnSForSubmittedProcessOnBothColosUsingColoHelper()
         throws Exception {
-        //schedule both bundles
-
         bundles[0].submitProcess(true);
-
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         bundles[1].submitProcess(true);
@@ -130,127 +139,109 @@ public class PrismProcessSnSTest extends BaseTestClass {
         bundles[1].submitProcess(true);
 
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
-        //now check if they have been scheduled correctly or not
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
-
     }
 
+    /**
+     * Submit and schedule process1 on cluster1 and check that only it is running there. Submit
+     * and schedule process2 on cluster2 and check the same for it. Schedule process1 on cluster2.
+     * Check that it is running on cluster2 and cluster1 but process2 isn't running on cluster1.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testProcessSnSAlreadyScheduledOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitAndScheduleProcess();
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        bundles[1].submitAndScheduleProcess();
 
-        //now check if they have been scheduled correctly or not
+        bundles[1].submitAndScheduleProcess();
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         //reschedule trial
-
         AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+            .schedule(URLS.SCHEDULE_URL, process1));
         Assert.assertEquals(OozieUtil.getBundles(cluster2.getFeedHelper().getOozieClient(),
-            Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).size(), 1);
+            Util.readEntityName(process1), EntityType.PROCESS).size(), 1);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
     }
 
+    /**
+     * Submit and schedule both process1 and process2. Suspend process1. Check their statuses.
+     * Submit and schedule process1 once more.
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testSnSSuspendedProcessOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitAndScheduleProcess();
         bundles[1].submitAndScheduleProcess();
-
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().suspend(URLS.SUSPEND_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        //now check if they have been scheduled correctly or not
+
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
-        Assert.assertEquals(OozieUtil.getBundles(cluster2.getFeedHelper().getOozieClient(),
-            Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).size(), 1);
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .resume(URLS.SUSPEND_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
+        Assert.assertEquals(OozieUtil.getBundles(cluster2OC, Util.readEntityName(process1),
+            EntityType.PROCESS).size(), 1);
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(URLS.SUSPEND_URL, process1));
 
-        AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(URLS.SUSPEND_URL, process2));
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getProcessData()));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2));
 
-        Assert.assertEquals(OozieUtil.getBundles(cluster1.getFeedHelper().getOozieClient(),
-            Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS).size(), 1);
+        Assert.assertEquals(OozieUtil.getBundles(cluster1OC, Util.readEntityName(process2),
+            EntityType.PROCESS).size(), 1);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
     }
 
+    /**
+     * Submit and schedule both processes on both cluster1 and cluster2. Check that they are
+     * running. Delete both of them. Submit and schedule them once more. Check that they are
+     * running again.
+     */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testSnSDeletedProcessOnBothColos() throws Exception {
         //schedule both bundles
         final String cluster1Running = cluster1.getClusterHelper().getColoName() + "/RUNNING";
         final String cluster2Running = cluster2.getClusterHelper().getColoName() + "/RUNNING";
-        bundles[0].submitAndScheduleProcess();
-
-        Assert.assertEquals(Util.parseResponse(
-                prism.getProcessHelper()
-                    .getStatus(URLS.STATUS_URL, bundles[0].getProcessData())).getMessage(),
-            cluster1Running
-        );
 
+        bundles[0].submitAndScheduleProcess();
+        Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
+            .getStatus(URLS.STATUS_URL, process1)).getMessage(), cluster1Running);
         bundles[1].submitAndScheduleProcess();
-        Assert.assertEquals(Util.parseResponse(
-                prism.getProcessHelper()
-                    .getStatus(URLS.STATUS_URL, bundles[1].getProcessData())).getMessage(),
-            cluster2Running
-        );
+        Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
+            .getStatus(URLS.STATUS_URL, process2)).getMessage(), cluster2Running);
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getProcessData()));
-
-        Assert.assertEquals(Util.parseResponse(
-                prism.getProcessHelper()
-                    .getStatus(URLS.STATUS_URL, bundles[0].getProcessData())
-            ).getMessage(),
-            cluster1Running
-        );
-        Assert.assertEquals(Util.parseResponse(
-                prism.getProcessHelper()
-                    .getStatus(URLS.STATUS_URL, bundles[1].getProcessData())
-            ).getMessage(),
-            cluster2Running
-        );
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2));
 
+        Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
+            .getStatus(URLS.STATUS_URL, process1)).getMessage(),cluster1Running);
+        Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
+            .getStatus(URLS.STATUS_URL, process2)).getMessage(), cluster2Running);
     }
 
+    /**
+     * Attempt to submit and schedule processes when all required entities weren't registered
+     */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testScheduleNonExistentProcessOnBothColos() throws Exception {
         Assert.assertEquals(Util.parseResponse(cluster2.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()))
-            .getStatusCode(), 404);
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1)).getStatusCode(), 404);
         Assert.assertEquals(Util.parseResponse(cluster1.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getProcessData()))
-            .getStatusCode(), 404);
-
-    }
-
-    @AfterClass(alwaysRun = true)
-    public void tearDownClass() throws IOException {
-        cleanTestDirs();
+            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2)).getStatusCode(), 404);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/3678eaba/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
index 458ddc6..c82786f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
@@ -176,27 +176,28 @@ public class ProcessUITest extends BaseUITestClass {
         //check Process statuses via UI
         EntitiesPage page = new EntitiesPage(DRIVER, cluster, EntityType.PROCESS);
         page.navigateTo();
-
-        softAssert.assertEquals(page.getEntityStatus(bundles[0].getProcessName()),
+        String process = bundles[0].getProcessData();
+        String processName = Util.readEntityName(process);
+        softAssert.assertEquals(page.getEntityStatus(processName),
                 EntitiesPage.EntityStatus.SUBMITTED, "Process status should be SUBMITTED");
-        prism.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL, process);
 
-        InstanceUtil.waitTillInstanceReachState(clusterOC, Util.readEntityName(bundles[0]
-                .getProcessData()), 1, CoordinatorAction.Status.RUNNING, EntityType.PROCESS);
+        InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 1,
+            CoordinatorAction.Status.RUNNING, EntityType.PROCESS);
 
-        softAssert.assertEquals(page.getEntityStatus(bundles[0].getProcessName()),
+        softAssert.assertEquals(page.getEntityStatus(processName),
                 EntitiesPage.EntityStatus.RUNNING, "Process status should be RUNNING");
 
-        ProcessPage processPage = new ProcessPage(DRIVER, cluster, bundles[0].getProcessName());
+        ProcessPage processPage = new ProcessPage(DRIVER, cluster, processName);
         processPage.navigateTo();
 
-        String bundleID = InstanceUtil.getLatestBundleID(cluster, bundles[0].getProcessName(), EntityType.PROCESS);
+        String bundleID = InstanceUtil.getLatestBundleID(cluster, processName, EntityType.PROCESS);
         Map<Date, CoordinatorAction.Status> actions = OozieUtil.getActionsNominalTimeAndStatus(prism, bundleID,
                 EntityType.PROCESS);
         checkActions(actions, processPage);
 
-        InstanceUtil.waitTillInstanceReachState(clusterOC, Util.readEntityName(bundles[0]
-                .getProcessData()), 1, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
+        InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 1,
+            CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
 
         processPage.refresh();
         actions = OozieUtil.getActionsNominalTimeAndStatus(prism, bundleID, EntityType.PROCESS);


[38/41] FALCON-675 Request URLS moved from parameters into methods in falcon-regression

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
index 1fdfa95..db82a8c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
@@ -27,7 +27,6 @@ import org.apache.falcon.regression.core.util.BundleUtil;
 import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.log4j.Logger;
@@ -85,9 +84,8 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
     private void submitFirstClusterScheduleFirstFeed()
         throws JAXBException, IOException, URISyntaxException, AuthenticationException {
         Assert.assertEquals(Util.parseResponse(prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0))).getStatusCode(), 200);
-        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(URLS
-            .SUBMIT_AND_SCHEDULE_URL, feed);
+            .submitEntity(bundles[0].getClusters().get(0))).getStatusCode(), 200);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
     }
 
@@ -107,8 +105,7 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
             .getLatestBundleID(cluster, Util.readEntityName(feed), EntityType.FEED);
 
         //try to submit and schedule the same process again
-        ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
@@ -124,8 +121,7 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void snsFeedWithoutCluster() throws Exception {
-        ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertFailed(response);
     }
 
@@ -139,11 +135,10 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
     public void snsDeletedFeed() throws Exception {
         submitFirstClusterScheduleFirstFeed();
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().delete(URLS.DELETE_URL,
-            feed)).getStatusCode(), 200);
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().delete(feed))
+            .getStatusCode(), 200);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.KILLED);
-        ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
     }
@@ -158,11 +153,10 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
     public void snsSuspendedFeed() throws Exception {
         submitFirstClusterScheduleFirstFeed();
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
-        Assert.assertEquals(Util.parseResponse(
-            prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed)).getStatusCode(), 200);
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().suspend(feed))
+            .getStatusCode(), 200);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
-        ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
index 72f4921..8d96741 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
@@ -24,7 +24,6 @@ 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.annotations.AfterClass;
@@ -54,7 +53,7 @@ public class FeedSubmitTest extends BaseTestClass {
 
         //submit the cluster
         ServiceResponse response =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+            prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         AssertUtil.assertSucceeded(response);
         feed = bundles[0].getInputFeedFromBundle();
     }
@@ -71,7 +70,7 @@ public class FeedSubmitTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void submitValidFeed() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
     }
 
@@ -82,13 +81,13 @@ public class FeedSubmitTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void submitValidFeedPostDeletion() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().delete(URLS.DELETE_URL, feed);
+        response = prism.getFeedHelper().delete(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
     }
 
@@ -99,13 +98,13 @@ public class FeedSubmitTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void submitValidFeedPostGet() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().getEntityDefinition(URLS.GET_ENTITY_DEFINITION, feed);
+        response = prism.getFeedHelper().getEntityDefinition(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
     }
 
@@ -116,10 +115,10 @@ public class FeedSubmitTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void submitValidFeedTwice() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
index 1973bf5..9676a30 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
@@ -24,7 +24,6 @@ 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -57,7 +56,7 @@ public class FeedSuspendTest extends BaseTestClass {
 
         //submit the cluster
         ServiceResponse response =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+            prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         AssertUtil.assertSucceeded(response);
 
         feed = bundles[0].getInputFeedFromBundle();
@@ -76,11 +75,10 @@ public class FeedSuspendTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void suspendScheduledFeed() throws Exception {
-        ServiceResponse response =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        response = prism.getFeedHelper().suspend(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
     }
@@ -93,14 +91,13 @@ public class FeedSuspendTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void suspendAlreadySuspendedFeed() throws Exception {
-        ServiceResponse response =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        response = prism.getFeedHelper().suspend(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
-        response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        response = prism.getFeedHelper().suspend(feed);
 
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
@@ -113,14 +110,13 @@ public class FeedSuspendTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void suspendDeletedFeed() throws Exception {
-        ServiceResponse response =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().delete(URLS.DELETE_URL, feed);
+        response = prism.getFeedHelper().delete(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        response = prism.getFeedHelper().suspend(feed);
         AssertUtil.assertFailed(response);
     }
 
@@ -131,7 +127,7 @@ public class FeedSuspendTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void suspendNonExistentFeed() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().suspend(URLS.SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().suspend(feed);
         AssertUtil.assertFailed(response);
     }
 
@@ -142,10 +138,10 @@ public class FeedSuspendTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void suspendSubmittedFeed() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        response = prism.getFeedHelper().suspend(feed);
         AssertUtil.assertFailed(response);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index 75eccaf..db600ff 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@ -255,7 +255,7 @@ public class InstanceSummaryTest extends BaseTestClass {
         feedInputPath, 1);*/
 
         //submit and schedule feed
-        prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(feed);
 
         InstancesSummaryResult r = prism.getFeedHelper()
             .getInstanceSummary(Util.readEntityName(feed), "?start=" + startTime);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
index c505a93..d4f31e9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
@@ -36,7 +36,6 @@ import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
@@ -114,14 +113,14 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessZeroAttemptUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -133,7 +132,7 @@ public class NewRetryTest extends BaseTestClass {
 
             //schedule process
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
 
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -167,13 +166,13 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessLowerAttemptUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -183,7 +182,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             //now wait till the process is over
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
 
@@ -225,13 +224,13 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessLowerManageableAttemptUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -241,7 +240,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -278,13 +277,13 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessLowerBoundaryAttemptUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -293,7 +292,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
 
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -334,13 +333,13 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -349,7 +348,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -383,13 +382,13 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessHigherDelayUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
         } else {
@@ -397,7 +396,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -432,15 +431,14 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessLowerDelayUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(
-                prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -449,7 +447,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -487,14 +485,14 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInProcessZeroDelayUpdate(Retry retry) throws Exception {
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         bundles[0].setRetry(retry);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -503,7 +501,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -540,13 +538,13 @@ public class NewRetryTest extends BaseTestClass {
         bundles[0].setRetry(retry);
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         bundles[0].setProcessLatePolicy(null);
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -555,7 +553,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -578,14 +576,14 @@ public class NewRetryTest extends BaseTestClass {
         bundles[0].setRetry(retry);
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         logger.info("process dates: " + startDate + "," + endDate);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -594,7 +592,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
 
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -632,14 +630,14 @@ public class NewRetryTest extends BaseTestClass {
         bundles[0].setRetry(retry);
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         logger.info("process dates: " + startDate + "," + endDate);
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -648,7 +646,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()),
@@ -689,12 +687,12 @@ public class NewRetryTest extends BaseTestClass {
         bundles[0].setRetry(retry);
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -703,7 +701,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
             List<DateTime> dates = null;
@@ -729,7 +727,7 @@ public class NewRetryTest extends BaseTestClass {
             logger.info("now suspending the process altogether....");
 
             AssertUtil.assertSucceeded(
-                cluster.getProcessHelper().suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
+                cluster.getProcessHelper().suspend(bundles[0].getProcessData()));
 
             HashMap<String, Integer> initialMap = getFailureRetriesForEachWorkflow(
                 clusterOC, getDefaultOozieCoordinator(clusterOC, bundleId));
@@ -760,7 +758,7 @@ public class NewRetryTest extends BaseTestClass {
 
             logger.info("now resuming the process...");
             AssertUtil.assertSucceeded(
-                cluster.getProcessHelper().resume(URLS.RESUME_URL, bundles[0].getProcessData()));
+                cluster.getProcessHelper().resume(bundles[0].getProcessData()));
 
             //now to validate all failed instances to check if they were retried or not.
             validateRetry(clusterOC, bundleId,
@@ -786,12 +784,12 @@ public class NewRetryTest extends BaseTestClass {
         bundles[0].setRetry(retry);
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
@@ -803,7 +801,7 @@ public class NewRetryTest extends BaseTestClass {
                 Util.getHadoopDataFromDir(clusterFS, bundles[0].getInputFeedFromBundle(),
                     lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
             List<DateTime> dates = null;
@@ -864,13 +862,13 @@ public class NewRetryTest extends BaseTestClass {
         bundles[0].setRetry(retry);
 
         for (String data : bundles[0].getDataSets()) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, data));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(data));
         }
 
 
         //submit and schedule process
         ServiceResponse response =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+            prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
         if (retry.getAttempts() <= 0 || retry.getDelay().getFrequencyAsInt() <= 0) {
             AssertUtil.assertFailed(response);
         } else {
@@ -878,7 +876,7 @@ public class NewRetryTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
             HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().schedule(bundles[0].getProcessData()));
             //now wait till the process is over
             String bundleId = OozieUtil.getBundles(clusterOC,
                 Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@ -887,7 +885,7 @@ public class NewRetryTest extends BaseTestClass {
                 (bundles[0].getProcessObject().getRetry().getAttempts()) / 2);
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper().delete(URLS.DELETE_URL, (bundles[0].getProcessData())));
+                prism.getProcessHelper().delete((bundles[0].getProcessData())));
 
             if (retry.getPolicy() == PolicyType.EXP_BACKOFF) {
                 TimeUnit.MINUTES.sleep(retry.getDelay().getFrequencyAsInt() * ((retry.getAttempts()

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
index 9c355a2..cf5ba0f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
@@ -32,7 +32,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -182,12 +181,11 @@ public class ProcessInstanceColoMixedTest extends BaseTestClass {
         LOGGER.info("feed02: " + Util.prettyPrintXml(feed02));
         LOGGER.info("outputFeed: " + Util.prettyPrintXml(outputFeed));
 
-        ServiceResponse r = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed01);
+        ServiceResponse r = prism.getFeedHelper().submitAndSchedule(feed01);
         AssertUtil.assertSucceeded(r);
-        r = prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed02);
+        r = prism.getFeedHelper().submitAndSchedule(feed02);
         AssertUtil.assertSucceeded(r);
-        r = prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, outputFeed);
+        r = prism.getFeedHelper().submitAndSchedule(outputFeed);
         AssertUtil.assertSucceeded(r);
 
         //create a process with 2 clusters
@@ -219,8 +217,7 @@ public class ProcessInstanceColoMixedTest extends BaseTestClass {
         //submit and schedule process
         LOGGER.info("process: " + Util.prettyPrintXml(process));
 
-        prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process);
+        prism.getProcessHelper().submitAndSchedule(process);
 
         LOGGER.info("Wait till process goes into running ");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
index bfb8d52..2e774c2 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
@@ -30,7 +30,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
@@ -218,7 +217,7 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeDeleted() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
         InstancesResult r = prism.getProcessHelper().getProcessInstanceResume(processName,
             "?start=2010-01-02T01:05Z");
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.PROCESS_NOT_FOUND);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
index e92f6b8..7a7e735 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
@@ -31,7 +31,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
@@ -110,12 +109,11 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         String process = bundles[0].getProcessData();
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(process));
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(URLS.RESUME_URL, process));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(process));
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
-            processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -129,11 +127,9 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     public void getSuspendedProcessInstance() throws Exception {
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
-            processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
@@ -146,8 +142,7 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void getRunningProcessInstance() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
-            processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -158,8 +153,7 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getNonExistenceProcessInstance() throws Exception {
-        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
-            "invalidName");
+        InstancesResult r = prism.getProcessHelper().getRunningInstance("invalidName");
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
@@ -172,9 +166,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void getKilledProcessInstance() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
-        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
-            processName);
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
@@ -189,8 +182,7 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     public void getSucceededProcessInstance() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
         InstanceUtil.waitForBundleToReachState(cluster, processName, Job.Status.SUCCEEDED);
-        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
-            processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
index 8c82da9..12a3907 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
@@ -32,7 +32,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
@@ -216,8 +215,7 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusKilled() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
         InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         if ((r.getStatusCode() != ResponseKeys.PROCESS_NOT_FOUND)) {
@@ -234,8 +232,7 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusOnlyStartSuspended() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         TimeUtil.sleepSeconds(TIMEOUT);
         InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z");
@@ -289,9 +286,9 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         String process = bundles[0].getProcessData();
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
-        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process);
+        prism.getProcessHelper().suspend(process);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
-        prism.getProcessHelper().resume(URLS.RESUME_URL, process);
+        prism.getProcessHelper().resume(process);
         TimeUtil.sleepSeconds(TIMEOUT);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
         InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
@@ -348,7 +345,7 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
         String process = bundles[0].getProcessData();
         bundles[0].submitFeedsScheduleProcess(prism);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(process));
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
         TimeUtil.sleepSeconds(TIMEOUT);
         InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
index 2a4f7f3..ebe6236 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
@@ -32,7 +32,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
@@ -241,11 +240,11 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
         TimeUtil.sleepSeconds(TIMEOUT);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        prism.getProcessHelper().getRunningInstance(processName);
         InstancesResult r = prism.getProcessHelper().getProcessInstanceSuspend(processName,
                 "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccessOnlyStart(r, WorkflowStatus.SUSPENDED);
-        prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        prism.getProcessHelper().getRunningInstance(processName);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
index 5d1f1b6..8b5c01c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
@@ -128,7 +128,7 @@ public class HCatFeedOperationsTest extends BaseTestClass {
         feed = bundles[1].getInputFeedFromBundle();
         FeedMerlin feedObj = new FeedMerlin(feed);
         feedObj.setTableValue(dbName, randomTblName, FeedType.YEARLY.getHcatPathValue());
-        ServiceResponse response = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feedObj.toString());
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedObj.toString());
         AssertUtil.assertFailed(response);
     }
 
@@ -144,13 +144,13 @@ public class HCatFeedOperationsTest extends BaseTestClass {
         feed = bundles[0].getInputFeedFromBundle();
         FeedMerlin feedObj = new FeedMerlin(feed);
         feedObj.setTableValue(dbName, tableName, FeedType.YEARLY.getHcatPathValue());
-        ServiceResponse response = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feedObj.toString());
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedObj.toString());
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().delete(Util.URLS.DELETE_URL, feedObj.toString());
+        response = prism.getFeedHelper().delete(feedObj.toString());
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feedObj.toString());
+        response = prism.getFeedHelper().submitEntity(feedObj.toString());
         AssertUtil.assertSucceeded(response);
     }
 
@@ -178,10 +178,7 @@ public class HCatFeedOperationsTest extends BaseTestClass {
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
                 tableUri);
 
-        AssertUtil.assertPartial(
-                prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                        feed)
-        );
+        AssertUtil.assertPartial(prism.getFeedHelper().submitAndSchedule(feed));
     }
 
     /**
@@ -209,10 +206,7 @@ public class HCatFeedOperationsTest extends BaseTestClass {
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
                 tableUri);
 
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                        feed)
-        );
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
         Assert.assertEquals(InstanceUtil
                 .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feed),
                         "REPLICATION"), 1);
@@ -231,19 +225,13 @@ public class HCatFeedOperationsTest extends BaseTestClass {
 
         submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
 
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL,
-                        feed)
-        );
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed));
 
         //check that feed suspended on both clusters
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.SUSPENDED);
 
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().resume(Util.URLS.RESUME_URL,
-                        feed)
-        );
+        AssertUtil.assertSucceeded(prism.getFeedHelper().resume(feed));
 
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.RUNNING);
@@ -259,10 +247,7 @@ public class HCatFeedOperationsTest extends BaseTestClass {
     public void deleteReplicationFeed() throws Exception {
         submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
 
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().delete(Util.URLS.DELETE_URL,
-                        feed)
-        );
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.KILLED);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
index 557a9f1..50ccd8a 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
@@ -182,10 +182,7 @@ public class HCatReplicationTest extends BaseTestClass {
             Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
             tableUri);
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                feed)
-        );
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
         TimeUtil.sleepSeconds(TIMEOUT);
         //check if all coordinators exist
         Assert.assertEquals(InstanceUtil
@@ -277,10 +274,7 @@ public class HCatReplicationTest extends BaseTestClass {
             Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.TARGET, null,
             tableUri);
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                feed)
-        );
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
         TimeUtil.sleepSeconds(TIMEOUT);
         //check if all coordinators exist
         Assert.assertEquals(InstanceUtil

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
index 5dc96e2..aae3125 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
@@ -32,7 +32,6 @@ import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.HadoopUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -105,7 +104,7 @@ public class HCatRetentionTest extends BaseTestClass {
             .setRetentionValue(retentionUnit.getValue() + "(" + retentionPeriod + ")");
         if (retentionPeriod <= 0) {
             AssertUtil.assertFailed(prism.getFeedHelper()
-                .submitEntity(URLS.SUBMIT_URL, bundle.getInputFeedFromBundle()));
+                .submitEntity(bundle.getInputFeedFromBundle()));
         } else {
             final DateTime dataStartTime = new DateTime(
                 feedElement.getClusters().getClusters().get(0).getValidity().getStart(),
@@ -127,12 +126,11 @@ public class HCatRetentionTest extends BaseTestClass {
             AssertUtil.checkForListSizes(initialData, initialPtnList);
 
             AssertUtil.assertSucceeded(prism.getFeedHelper()
-                .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedElement.toString()));
+                .submitAndSchedule(feedElement.toString()));
             final String bundleId = OozieUtil.getBundles(clusterOC, feedElement.getName(),
                 EntityType.FEED).get(0);
             OozieUtil.waitForRetentionWorkflowToSucceed(bundleId, clusterOC);
-            AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(URLS.SUSPEND_URL,
-                feedElement.toString()));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feedElement.toString()));
 
             List<String> expectedOutput = getExpectedOutput(retentionPeriod, retentionUnit,
                 feedType, new DateTime(DateTimeZone.UTC), initialData);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
index 0c6ecae..143789f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
@@ -38,7 +38,6 @@ import org.apache.falcon.regression.core.util.BundleUtil;
 import org.apache.falcon.regression.core.util.CleanupUtil;
 import org.apache.falcon.regression.core.util.Generator;
 import org.apache.falcon.regression.core.util.GraphAssert;
-import org.apache.falcon.regression.core.util.Util;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.http.HttpResponse;
 import org.apache.log4j.Logger;
@@ -91,8 +90,7 @@ public class LineageApiTest extends BaseTestClass {
         Assert.assertEquals(clusterStrings.size(), 1, "Expecting only 1 clusterMerlin.");
         clusterMerlin = new ClusterMerlin(clusterStrings.get(0));
         clusterMerlin.setTags(testTag);
-        AssertUtil.assertSucceeded(
-            prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL, clusterMerlin.toString()));
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(clusterMerlin.toString()));
         logger.info("numInputFeeds = " + numInputFeeds);
         logger.info("numOutputFeeds = " + numOutputFeeds);
         final FeedMerlin inputMerlin = new FeedMerlin(bundles[0].getInputFeedFromBundle());
@@ -101,8 +99,7 @@ public class LineageApiTest extends BaseTestClass {
             Generator.getNameGenerator("infeed", inputMerlin.getName()),
             Generator.getHadoopPathGenerator(feedInputPath, MINUTE_DATE_PATTERN));
         for (FeedMerlin feed : inputFeeds) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL,
-                feed.toString()));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed.toString()));
         }
 
         FeedMerlin outputMerlin = new FeedMerlin(bundles[0].getOutputFeedFromBundle());
@@ -111,8 +108,7 @@ public class LineageApiTest extends BaseTestClass {
             Generator.getNameGenerator("outfeed", outputMerlin.getName()),
             Generator.getHadoopPathGenerator(feedOutputPath, MINUTE_DATE_PATTERN));
         for (FeedMerlin feed : outputFeeds) {
-            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL,
-                feed.toString()));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed.toString()));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
index 72d03cd..55d69a3 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
@@ -89,8 +89,7 @@ public class EntityDryRunTest extends BaseTestClass {
     public void testDryRunFailureScheduleProcess() throws Exception {
         bundles[0].setProcessProperty("EntityDryRunTestProp", "${coord:someEL(1)");
         bundles[0].submitProcess(true);
-        ServiceResponse response = prism.getProcessHelper()
-            .schedule(Util.URLS.SCHEDULE_URL, bundles[0].getProcessData());
+        ServiceResponse response = prism.getProcessHelper().schedule(bundles[0].getProcessData());
         validate(response);
     }
 
@@ -119,8 +118,7 @@ public class EntityDryRunTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         feed = Util.setFeedProperty(feed, "EntityDryRunTestProp", "${coord:someEL(1)");
         bundles[0].submitClusters(prism);
-        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(
-            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         validate(response);
     }
 
@@ -131,8 +129,7 @@ public class EntityDryRunTest extends BaseTestClass {
     public void testDryRunFailureUpdateFeed() throws Exception {
         bundles[0].submitClusters(prism);
         String feed = bundles[0].getInputFeedFromBundle();
-        ServiceResponse response =
-            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
         feed = Util.setFeedProperty(feed, "EntityDryRunTestProp", "${coord:someEL(1)");
         response = prism.getFeedHelper().update(feed, feed);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
index c234f0d..6298981 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
@@ -26,7 +26,6 @@ import org.apache.falcon.regression.core.util.BundleUtil;
 import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -117,8 +116,7 @@ public class FeedDelayParallelTimeoutTest extends BaseTestClass {
         feedOutput01 = Util.setFeedProperty(feedOutput01, "parallel", "3");
 
         logger.info("feedOutput01: " + Util.prettyPrintXml(feedOutput01));
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedOutput01);
+        prism.getFeedHelper().submitAndSchedule(feedOutput01);
     }
 
     @AfterClass(alwaysRun = true)


[05/41] git commit: FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored contributed by Paul Isaychuk

Posted by ra...@apache.org.
FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored contributed by Paul Isaychuk


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

Branch: refs/heads/FALCON-585
Commit: 78b9c1a91b698d78d881eb5555352ebc4b259e85
Parents: 1db8712
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Thu Aug 28 11:25:12 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Thu Aug 28 11:25:12 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   3 +
 .../falcon/regression/InstanceSummaryTest.java  |   4 +-
 .../prism/UpdateAtSpecificTimeTest.java         | 441 +++++++------------
 3 files changed, 167 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/78b9c1a9/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 1357808..5ed6a89 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -8,6 +8,9 @@ Trunk (Unreleased)
    FALCON-589 Add test cases for various feed operations on Hcat feeds (Karishma G 
    via Samarth Gupta)
   IMPROVEMENTS
+   FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored
+   (Paul Isaychuk via Samarth Gupta)
+
    FALCON-619 ELExp_FutureAndLatestTest stabilization (Paul Isaychuk via Arpit Gupta)
 
    FALCON-610 Refactoring and documentation updates (Paul Isaychuk via Arpit Gupta)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/78b9c1a9/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index c541620..9901fb1 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@ -53,6 +53,7 @@ import java.util.List;
 this test currently provide minimum verification. More detailed test should
  be added
  */
+@Test(groups = "embedded")
 public class InstanceSummaryTest extends BaseTestClass {
 
     //1. process : test summary single cluster few instance some future some past
@@ -304,8 +305,5 @@ public class InstanceSummaryTest extends BaseTestClass {
     public void tearDown() throws IOException {
         processBundle.deleteBundle(prism);
         removeBundles();
-        for (FileSystem fs : serverFS) {
-            HadoopUtil.deleteDirIfExists(Util.getPathPrefix(feedInputPath), fs);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/78b9c1a9/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
index a017c79..8bfeda4 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
@@ -42,6 +42,7 @@ import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClientException;
 import org.custommonkey.xmlunit.Diff;
 import org.custommonkey.xmlunit.XMLUnit;
+import org.joda.time.DateTime;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
@@ -62,12 +63,10 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
     private static final Logger logger = Logger.getLogger(UpdateAtSpecificTimeTest.class);
 
     Bundle processBundle;
-
-    ColoHelper cluster_1 = servers.get(0);
-    ColoHelper cluster_2 = servers.get(1);
-    ColoHelper cluster_3 = servers.get(2);
+    ColoHelper cluster1 = servers.get(0);
+    ColoHelper cluster2 = servers.get(1);
+    ColoHelper cluster3 = servers.get(2);
     FileSystem cluster2FS = serverFS.get(1);
-
     private String dateTemplate = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private final String baseTestDir = baseHDFSDir + "/UpdateAtSpecificTimeTest-data";
     String aggregateWorkflowDir = baseHDFSDir + "/aggregator";
@@ -77,21 +76,20 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
     }
 
-
     @BeforeMethod(alwaysRun = true)
     public void setup(Method method) throws IOException {
         logger.info("test name: " + method.getName());
         Bundle bundle = BundleUtil.readLocalDCBundle();
-        bundles[0] = new Bundle(bundle, cluster_1);
-        bundles[1] = new Bundle(bundle, cluster_2);
-        bundles[2] = new Bundle(bundle, cluster_3);
+        bundles[0] = new Bundle(bundle, cluster1);
+        bundles[1] = new Bundle(bundle, cluster2);
+        bundles[2] = new Bundle(bundle, cluster3);
 
         bundles[0].generateUniqueBundle();
         bundles[1].generateUniqueBundle();
         bundles[2].generateUniqueBundle();
 
         processBundle = BundleUtil.readELBundle();
-        processBundle = new Bundle(processBundle, cluster_1);
+        processBundle = new Bundle(processBundle, cluster1);
         processBundle.generateUniqueBundle();
         processBundle.setProcessWorkflow(aggregateWorkflowDir);
     }
@@ -102,15 +100,14 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         removeBundles(processBundle);
     }
 
-    @Test(groups = {"singleCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"singleCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void invalidChar_Process()
         throws JAXBException, IOException, URISyntaxException,
         AuthenticationException, OozieClientException {
         processBundle.setProcessValidity(TimeUtil.getTimeWrtSystemTime(0),
             TimeUtil.getTimeWrtSystemTime(20));
         processBundle.submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, processBundle.getProcessData(), 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 0);
         String oldProcess =
             processBundle.getProcessData();
         processBundle.setProcessValidity(TimeUtil.getTimeWrtSystemTime(5),
@@ -121,26 +118,23 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             .contains("java.lang.IllegalArgumentException: abc is not a valid UTC string"));
     }
 
-    @Test(groups = {"singleCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"singleCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void invalidChar_Feed()
         throws JAXBException, IOException, URISyntaxException, AuthenticationException,
         OozieClientException {
 
         String feed = submitAndScheduleFeed(processBundle);
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, feed, 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
+
         //update frequency
         Frequency f = new Frequency("" + 21, Frequency.TimeUnit.minutes);
         String updatedFeed = InstanceUtil.setFeedFrequency(feed, f);
-
         ServiceResponse r = prism.getFeedHelper().update(feed, updatedFeed, "abc", null);
         Assert.assertTrue(r.getMessage()
             .contains("java.lang.IllegalArgumentException: abc is not a valid UTC string"));
     }
 
-
-    @Test(groups = {"singleCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"singleCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void updateTimeInPast_Process()
         throws JAXBException, IOException, URISyntaxException,
         OozieClientException, AuthenticationException {
@@ -151,49 +145,35 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
 
         //get old process details
         String oldProcess = processBundle.getProcessData();
+        String oldBundleId = InstanceUtil.getLatestBundleID(cluster1,
+            Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
 
-        String oldBundleId = InstanceUtil
-            .getLatestBundleID(cluster_1,
-                Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
-
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, oldProcess, 0);
-
-        List<String> initialNominalTimes = OozieUtil.getActionsNominalTime(cluster_1,
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, oldProcess, 0);
+        List<String> initialNominalTimes = OozieUtil.getActionsNominalTime(cluster1,
             oldBundleId, EntityType.PROCESS);
 
-
         // update process by adding property
         processBundle.setProcessProperty("someProp", "someValue");
         ServiceResponse r = prism.getProcessHelper().update(oldProcess,
-            processBundle.getProcessData(),TimeUtil.getTimeWrtSystemTime(-10000), null);
+            processBundle.getProcessData(), TimeUtil.getTimeWrtSystemTime(-10000), null);
         AssertUtil.assertSucceeded(r);
 
         //check new coord created with current time
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleId, initialNominalTimes,
-            processBundle.getProcessData(), true,
-            false);
-
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, oldProcess, 1);
-
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleId, initialNominalTimes,
-            processBundle.getProcessData(), true,
-            true);
-
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleId, initialNominalTimes,
+            processBundle.getProcessData(), true, false);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, oldProcess, 1);
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleId, initialNominalTimes,
+            processBundle.getProcessData(), true, true);
     }
 
-    @Test(groups = {"MultiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
-
+    @Test(groups = {"MultiCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void updateTimeInPast_Feed()
         throws JAXBException, IOException, OozieClientException,
         URISyntaxException, AuthenticationException {
 
-
         String startTimeCluster_source = TimeUtil.getTimeWrtSystemTime(-10);
         String startTimeCluster_target = TimeUtil.getTimeWrtSystemTime(10);
-
         String feed = getMultiClusterFeed(startTimeCluster_source, startTimeCluster_target);
-
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
         //submit and schedule feed
@@ -201,54 +181,47 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
-
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, feed, 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
 
         //update frequency
         Frequency f = new Frequency("" + 7, Frequency.TimeUnit.minutes);
         String updatedFeed = InstanceUtil.setFeedFrequency(feed, f);
-
         r = prism.getFeedHelper().update(feed, updatedFeed,
             TimeUtil.getTimeWrtSystemTime(-10000), null);
         AssertUtil.assertSucceeded(r);
-
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, feed, 1);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 1);
 
         //check correct number of coord exists or not
         Assert.assertEquals(InstanceUtil
-            .checkIfFeedCoordExist(cluster_1.getFeedHelper(),
+            .checkIfFeedCoordExist(cluster1.getFeedHelper(),
                 Util.readEntityName(feed),
                 "REPLICATION"), 2);
         Assert.assertEquals(InstanceUtil
-            .checkIfFeedCoordExist(cluster_2.getFeedHelper(), Util.readEntityName(feed),
+            .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feed),
                 "RETENTION"), 2);
         Assert.assertEquals(InstanceUtil
-            .checkIfFeedCoordExist(cluster_1.getFeedHelper(), Util.readEntityName(feed),
+            .checkIfFeedCoordExist(cluster1.getFeedHelper(), Util.readEntityName(feed),
                 "RETENTION"), 2);
         Assert.assertEquals(InstanceUtil
-            .checkIfFeedCoordExist(cluster_3.getFeedHelper(), Util.readEntityName(feed),
+            .checkIfFeedCoordExist(cluster3.getFeedHelper(), Util.readEntityName(feed),
                 "RETENTION"), 2);
-
     }
 
-
-    @Test(groups = {"MultiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"MultiCluster", "0.3.1", "distributed"}, timeOut = 1200000, enabled = true)
     public void inNextFewMinutesUpdate_RollForward_Process()
         throws JAXBException, IOException, URISyntaxException, JSchException,
         OozieClientException, SAXException, AuthenticationException {
-    /*
-    submit process on 3 clusters. Schedule on 2 clusters. Bring down one of
-    the scheduled cluster. Update with time 5 minutes from now. On running
-    cluster new coord should be created with start time +5 and no instance
-    should be missing. On 3rd cluster where process was only submit,
-    definition should be updated. Bring the down cluster up. Update with same
-     definition again, now the recently up cluster should also have new
-     coords.
-     */
-
+        /*
+        submit process on 3 clusters. Schedule on 2 clusters. Bring down one of
+        the scheduled cluster. Update with time 5 minutes from now. On running
+        cluster new coord should be created with start time +5 and no instance
+        should be missing. On 3rd cluster where process was only submit,
+        definition should be updated. Bring the down cluster up. Update with same
+        definition again, now the recently up cluster should also have new
+        coords.
+        */
         try {
-            Util.startService(cluster_2.getProcessHelper());
+            Util.startService(cluster2.getProcessHelper());
             String startTime = TimeUtil.getTimeWrtSystemTime(-15);
             processBundle.setProcessValidity(startTime,
                 TimeUtil.getTimeWrtSystemTime(60));
@@ -259,32 +232,28 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             processBundle.submitBundle(prism);
 
             //schedule of 2 cluster
-            cluster_1.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL,
+            cluster1.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL,
                 processBundle.getProcessData());
-
-            cluster_2.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL,
+            cluster2.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL,
                 processBundle.getProcessData());
+            InstanceUtil.waitTillInstancesAreCreated(cluster2, processBundle.getProcessData(), 0);
 
-            InstanceUtil.waitTillInstancesAreCreated(cluster_2, processBundle.getProcessData(), 0);
-
-            //shut down cluster_2
-            Util.shutDownService(cluster_2.getProcessHelper());
+            //shut down cluster2
+            Util.shutDownService(cluster2.getProcessHelper());
 
             // save old data before update
             String oldProcess = processBundle.getProcessData();
             String oldBundleID_cluster1 = InstanceUtil
-                .getLatestBundleID(cluster_1,
+                .getLatestBundleID(cluster1,
                     Util.readEntityName(oldProcess), EntityType.PROCESS);
             String oldBundleID_cluster2 = InstanceUtil
-                .getLatestBundleID(cluster_2,
+                .getLatestBundleID(cluster2,
                     Util.readEntityName(oldProcess), EntityType.PROCESS);
-
             List<String> oldNominalTimes_cluster1 = OozieUtil.getActionsNominalTime
-                (cluster_1,
+                (cluster1,
                     oldBundleID_cluster1, EntityType.PROCESS);
-
             List<String> oldNominalTimes_cluster2 = OozieUtil.getActionsNominalTime
-                (cluster_2,
+                (cluster2,
                     oldBundleID_cluster2, EntityType.PROCESS);
 
             //update process validity
@@ -293,219 +262,172 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             //send update request
             String updateTime = TimeUtil.getTimeWrtSystemTime(5);
             ServiceResponse r = prism.getProcessHelper()
-                .update(oldProcess, processBundle.getProcessData(), updateTime
-                );
+                .update(oldProcess, processBundle.getProcessData(), updateTime);
             AssertUtil.assertPartial(r);
+            InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 1);
 
-            InstanceUtil.waitTillInstancesAreCreated(cluster_1, processBundle.getProcessData(), 1);
-
-            //verify new bundle on cluster_1 and definition on cluster_3
+            //verify new bundle on cluster1 and definition on cluster3
             OozieUtil
-                .verifyNewBundleCreation(cluster_1, oldBundleID_cluster1, oldNominalTimes_cluster1,
+                .verifyNewBundleCreation(cluster1, oldBundleID_cluster1, oldNominalTimes_cluster1,
                     oldProcess, true, false);
-
-            OozieUtil.verifyNewBundleCreation(cluster_2, oldBundleID_cluster2,
+            OozieUtil.verifyNewBundleCreation(cluster2, oldBundleID_cluster2,
                 oldNominalTimes_cluster2,
                 oldProcess, false, false);
-
-            String definition_cluster_3 = Util.getEntityDefinition(cluster_3,
+            String definition_cluster_3 = Util.getEntityDefinition(cluster3,
                 processBundle.getProcessData(), true);
-
             Assert.assertTrue(XmlUtil.isIdentical(definition_cluster_3,
                 processBundle.getProcessData()), "Process definitions should be equal");
 
-            //start the stopped cluster_2
-            Util.startService(cluster_2.getProcessHelper());
+            //start the stopped cluster2
+            Util.startService(cluster2.getProcessHelper());
             TimeUtil.sleepSeconds(40);
-
-            String newBundleID_cluster1 = InstanceUtil
-                .getLatestBundleID(cluster_1,
-                    Util.readEntityName(oldProcess), EntityType.PROCESS);
+            String newBundleID_cluster1 = InstanceUtil.getLatestBundleID(cluster1,
+                Util.readEntityName(oldProcess), EntityType.PROCESS);
 
             //send second update request
-            r = prism.getProcessHelper().update(oldProcess,
-                processBundle.getProcessData(),
+            r = prism.getProcessHelper().update(oldProcess, processBundle.getProcessData(),
                 updateTime, null);
             AssertUtil.assertSucceeded(r);
-
-
-            String def_cluster_2 = Util.getEntityDefinition(cluster_2,
+            String def_cluster_2 = Util.getEntityDefinition(cluster2,
                 processBundle.getProcessData(), true);
             logger.info("def_cluster_2 : " + Util.prettyPrintXml(def_cluster_2));
 
-            // verify new bundle in cluster_2 and no new bundle in cluster_1  and
-            OozieUtil
-                .verifyNewBundleCreation(cluster_1, newBundleID_cluster1, oldNominalTimes_cluster1,
-                    oldProcess, false, false);
-
-            OozieUtil.verifyNewBundleCreation(cluster_2, oldBundleID_cluster2,
-                oldNominalTimes_cluster2,
-                oldProcess, true, false);
+            // verify new bundle in cluster2 and no new bundle in cluster1  and
+            OozieUtil.verifyNewBundleCreation(cluster1, newBundleID_cluster1,
+                oldNominalTimes_cluster1, oldProcess, false, false);
+            OozieUtil.verifyNewBundleCreation(cluster2, oldBundleID_cluster2,
+                oldNominalTimes_cluster2, oldProcess, true, false);
 
             //wait till update time is reached
             TimeUtil.sleepTill(updateTime);
-
-            OozieUtil.verifyNewBundleCreation(cluster_2, oldBundleID_cluster2,
-                oldNominalTimes_cluster2,
-                oldProcess, true, true);
-
-            OozieUtil
-                .verifyNewBundleCreation(cluster_1, oldBundleID_cluster1, oldNominalTimes_cluster1,
-                    oldProcess, true, true);
+            OozieUtil.verifyNewBundleCreation(cluster2, oldBundleID_cluster2,
+                oldNominalTimes_cluster2, oldProcess, true, true);
+            OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID_cluster1,
+                oldNominalTimes_cluster1, oldProcess, true, true);
         } finally {
-            Util.restartService(cluster_2.getProcessHelper());
+            Util.restartService(cluster2.getProcessHelper());
         }
     }
 
-    @Test(groups = {"MultiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"MultiCluster", "0.3.1", "distributed"}, timeOut = 1200000, enabled = true)
     public void inNextFewMinutesUpdate_RollForward_Feed()
         throws JAXBException, IOException, URISyntaxException, JSchException, 
         OozieClientException, SAXException, AuthenticationException {
         try {
             String startTimeCluster_source = TimeUtil.getTimeWrtSystemTime(-18);
-
             String feed = getMultiClusterFeed(startTimeCluster_source, startTimeCluster_source);
-
             logger.info("feed: " + Util.prettyPrintXml(feed));
 
             //submit feed on all 3 clusters
             ServiceResponse r = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feed);
             AssertUtil.assertSucceeded(r);
 
-            //schedule feed of cluster_1 and cluster_2
-            r = cluster_1.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed);
+            //schedule feed of cluster1 and cluster2
+            r = cluster1.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed);
             AssertUtil.assertSucceeded(r);
-            r = cluster_2.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed);
+            r = cluster2.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed);
             AssertUtil.assertSucceeded(r);
+            InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
 
-            InstanceUtil.waitTillInstancesAreCreated(cluster_1, feed, 0);
-
-            //shutdown cluster_2
-            Util.shutDownService(cluster_2.getProcessHelper());
+            //shutdown cluster2
+            Util.shutDownService(cluster2.getProcessHelper());
 
             //add some property to feed so that new bundle is created
             String updatedFeed = Util.setFeedProperty(feed, "someProp", "someVal");
 
             //save old data
-            String oldBundle_cluster1 = InstanceUtil.getLatestBundleID(cluster_1,
+            String oldBundle_cluster1 = InstanceUtil.getLatestBundleID(cluster1,
                 Util.readEntityName(feed), EntityType.FEED);
-
             List<String> oldNominalTimes_cluster1 = OozieUtil.getActionsNominalTime
-                (cluster_1,
-                    oldBundle_cluster1, EntityType.FEED);
+                (cluster1, oldBundle_cluster1, EntityType.FEED);
 
             //send update command with +5 mins in future
             String updateTime = TimeUtil.getTimeWrtSystemTime(5);
             r = prism.getFeedHelper().update(feed, updatedFeed, updateTime, null);
             AssertUtil.assertPartial(r);
 
-            //verify new bundle creation on cluster_1 and new definition on cluster_3
-            OozieUtil
-                .verifyNewBundleCreation(cluster_1, oldBundle_cluster1, oldNominalTimes_cluster1,
-                    feed, true, false);
-
-
-            String definition = Util.getEntityDefinition(cluster_3, feed, true);
+            //verify new bundle creation on cluster1 and new definition on cluster3
+            OozieUtil.verifyNewBundleCreation(cluster1, oldBundle_cluster1,
+                oldNominalTimes_cluster1, feed, true, false);
+            String definition = Util.getEntityDefinition(cluster3, feed, true);
             Diff diff = XMLUnit.compareXML(definition, processBundle.getProcessData());
             logger.info(diff);
 
-            //start stopped cluster_2
-            Util.startService(cluster_2.getProcessHelper());
-
-            String newBundle_cluster1 = InstanceUtil.getLatestBundleID(cluster_1,
+            //start stopped cluster2
+            Util.startService(cluster2.getProcessHelper());
+            String newBundle_cluster1 = InstanceUtil.getLatestBundleID(cluster1,
                 Util.readEntityName(feed), EntityType.FEED);
 
             //send update again
             r = prism.getFeedHelper().update(feed, updatedFeed, updateTime, null);
             AssertUtil.assertSucceeded(r);
 
-            //verify new bundle creation on cluster_2 and no new bundle on cluster_1
+            //verify new bundle creation on cluster2 and no new bundle on cluster1
             Assert.assertEquals(InstanceUtil
-                .checkIfFeedCoordExist(cluster_2.getFeedHelper(), Util.readEntityName(feed),
+                .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feed),
                     "RETENTION"), 2);
+            OozieUtil.verifyNewBundleCreation(cluster1, newBundle_cluster1,
+                oldNominalTimes_cluster1, feed, false, false);
 
-            OozieUtil
-                .verifyNewBundleCreation(cluster_1, newBundle_cluster1, oldNominalTimes_cluster1,
-                    feed, false, false);
             //wait till update time is reached
             TimeUtil.sleepTill(TimeUtil.getTimeWrtSystemTime(5));
 
             //verify new bundle creation with instance matching
-            OozieUtil
-                .verifyNewBundleCreation(cluster_1, oldBundle_cluster1, oldNominalTimes_cluster1,
-                    feed, true, true);
-
+            OozieUtil.verifyNewBundleCreation(cluster1, oldBundle_cluster1,
+                oldNominalTimes_cluster1, feed, true, true);
         } finally {
-            Util.restartService(cluster_2.getProcessHelper());
+            Util.restartService(cluster2.getProcessHelper());
         }
     }
 
-
-    @Test(groups = {"multiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"multiCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void updateTimeAfterEndTime_Process()
         throws JAXBException, InterruptedException, IOException, URISyntaxException,
         OozieClientException, AuthenticationException {
-
-    /*
-      submit and schedule process with end time after 60 mins. Set update time
-       as with +60 from start mins.
-    */
+        /* submit and schedule process with end time after 60 mins. Set update time
+           as with +60 from start mins */
         logger.info("Running test updateTimeAfterEndTime_Process");
         String startTime = TimeUtil.getTimeWrtSystemTime(-15);
         String endTime = TimeUtil.getTimeWrtSystemTime(60);
         processBundle.setProcessValidity(startTime, endTime);
         processBundle.submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(10);
-
         InstanceUtil.waitTillInstanceReachState(serverOC.get(0),
             Util.readEntityName(processBundle.getProcessData()), 0,
             CoordinatorAction.Status.WAITING, EntityType.PROCESS);
 
         //save old data
         String oldProcess = processBundle.getProcessData();
-
         String oldBundleID = InstanceUtil
-            .getLatestBundleID(cluster_1,
+            .getLatestBundleID(cluster1,
                 Util.readEntityName(oldProcess), EntityType.PROCESS);
-
-        List<String> oldNominalTimes = OozieUtil.getActionsNominalTime(cluster_1, oldBundleID,
+        List<String> oldNominalTimes = OozieUtil.getActionsNominalTime(cluster1, oldBundleID,
             EntityType.PROCESS);
 
         //update
         processBundle.setProcessProperty("someProp", "someVal");
         String updateTime = TimeUtil.addMinsToTime(endTime, 60);
-
         logger.info("Original Feed : " + Util.prettyPrintXml(oldProcess));
         logger.info("Updated Feed :" + Util.prettyPrintXml(processBundle.getProcessData()));
         logger.info("Update Time : " + updateTime);
-
-
         ServiceResponse r = prism.getProcessHelper().update(oldProcess,
             processBundle.getProcessData(), updateTime, null);
         AssertUtil.assertSucceeded(r);
 
         //verify new bundle creation with instances matching
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleID, oldNominalTimes,
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID, oldNominalTimes,
             oldProcess, true, false);
-
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, processBundle.getProcessData(), 1);
-
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleID, oldNominalTimes,
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 1);
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID, oldNominalTimes,
             oldProcess, true, true);
     }
 
-    @Test(groups = {"multiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"multiCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void updateTimeAfterEndTime_Feed()
         throws JAXBException, IOException, OozieClientException,
         URISyntaxException, AuthenticationException {
-    /*
-    submit and schedule feed with end time 60 mins in future and update with
-    +60
-     in future.
-     */
+
+        /* submit and schedule feed with end time 60 mins in future and update with +60 in future*/
         String startTime = TimeUtil.getTimeWrtSystemTime(-15);
         String endTime = TimeUtil.getTimeWrtSystemTime(60);
 
@@ -514,79 +436,64 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             XmlUtil.createValidity("2012-10-01T12:00Z", "2010-01-01T00:00Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE), null,
             ClusterType.SOURCE, null);
-
         feed = InstanceUtil.setFeedCluster(feed, XmlUtil.createValidity(startTime, endTime),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
             Util.readEntityName(processBundle.getClusters().get(0)), ClusterType.SOURCE,
             null, baseTestDir + "/replication" + dateTemplate);
 
-
         ServiceResponse r = prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
             processBundle.getClusters().get(0));
         AssertUtil.assertSucceeded(r);
-        r = prism.getFeedHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(r);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
 
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, feed, 0);
         //save old data
-
-        String oldBundleID = InstanceUtil
-            .getLatestBundleID(cluster_1,
-                Util.readEntityName(feed), EntityType.FEED);
-
+        String oldBundleID = InstanceUtil.getLatestBundleID(cluster1,
+            Util.readEntityName(feed), EntityType.FEED);
         String updateTime = TimeUtil.addMinsToTime(endTime, 60);
         String updatedFeed = Util.setFeedProperty(feed, "someProp", "someVal");
-
         logger.info("Original Feed : " + Util.prettyPrintXml(feed));
         logger.info("Updated Feed :" + Util.prettyPrintXml(updatedFeed));
         logger.info("Update Time : " + updateTime);
-
         r = prism.getFeedHelper().update(feed, updatedFeed, updateTime, null);
         AssertUtil.assertSucceeded(r);
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, feed, 1);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 1);
 
         //verify new bundle creation
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleID, null,
-            feed, true, false);
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID, null, feed, true, false);
     }
 
-    @Test(groups = {"multiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"multiCluster", "0.3.1", "embedded"}, timeOut = 1200000, enabled = true)
     public void updateTimeBeforeStartTime_Process() throws JAXBException, IOException,
         URISyntaxException, OozieClientException, AuthenticationException {
 
-    /*
-      submit and schedule process with start time +10 mins from now. Update
-      with start time -4 and update time +2 mins
-     */
+        /* submit and schedule process with start time +10 mins from now. Update with start time
+        -4 and update time +2 mins */
         String startTime = TimeUtil.getTimeWrtSystemTime(10);
         String endTime = TimeUtil.getTimeWrtSystemTime(20);
         processBundle.setProcessValidity(startTime, endTime);
         processBundle.submitFeedsScheduleProcess(prism);
+
         //save old data
         String oldProcess = processBundle.getProcessData();
-        String oldBundleID = InstanceUtil
-            .getLatestBundleID(cluster_1,
-                Util.readEntityName(oldProcess), EntityType.PROCESS);
-        List<String> oldNominalTimes = OozieUtil.getActionsNominalTime(cluster_1, oldBundleID,
+        String oldBundleID = InstanceUtil.getLatestBundleID(cluster1,
+            Util.readEntityName(oldProcess), EntityType.PROCESS);
+        List<String> oldNominalTimes = OozieUtil.getActionsNominalTime(cluster1, oldBundleID,
             EntityType.PROCESS);
-
-        processBundle.setProcessValidity(TimeUtil.addMinsToTime(startTime, -4),
-            endTime);
+        processBundle.setProcessValidity(TimeUtil.addMinsToTime(startTime, -4), endTime);
         String updateTime = TimeUtil.getTimeWrtSystemTime(2);
         ServiceResponse r = prism.getProcessHelper().update(oldProcess,
             processBundle.getProcessData(), updateTime, null);
         AssertUtil.assertSucceeded(r);
         TimeUtil.sleepSeconds(10);
+
         //verify new bundle creation
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleID, oldNominalTimes,
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID, oldNominalTimes,
             oldProcess, true, false);
-
     }
 
-    @Test(groups = {"MultiCluster", "0.3.1"}, timeOut = 1200000,
-        enabled = true)
+    @Test(groups = {"MultiCluster", "0.3.1"}, timeOut = 1200000, enabled = true)
     public void updateDiffClusterDiffValidity_Process()
         throws JAXBException, IOException, URISyntaxException, OozieClientException,
         AuthenticationException {
@@ -599,10 +506,8 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         String startTime_cluster3 = TimeUtil.getTimeWrtSystemTime(-30);
         String endTime_cluster3 = TimeUtil.getTimeWrtSystemTime(180);
 
-
         //create multi cluster bundle
-        processBundle.setProcessValidity(startTime_cluster1,
-            endTime_cluster1);
+        processBundle.setProcessValidity(startTime_cluster1, endTime_cluster1);
         processBundle.addClusterToBundle(bundles[1].getClusters().get(0),
             ClusterType.SOURCE, startTime_cluster2, endTime_cluster2);
         processBundle.addClusterToBundle(bundles[2].getClusters().get(0),
@@ -612,26 +517,20 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         processBundle.submitFeedsScheduleProcess(prism);
 
         //wait for coord to be in running state
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, processBundle.getProcessData(), 0);
-        InstanceUtil.waitTillInstancesAreCreated(cluster_3, processBundle.getProcessData(), 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 0);
+        InstanceUtil.waitTillInstancesAreCreated(cluster3, processBundle.getProcessData(), 0);
 
         //save old info
-        String oldBundleID_cluster1 = InstanceUtil
-            .getLatestBundleID(cluster_1,
-                Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
-        List<String> nominalTimes_cluster1 =
-            OozieUtil.getActionsNominalTime(cluster_1, oldBundleID_cluster1,
-                EntityType.PROCESS);
-        String oldBundleID_cluster2 = InstanceUtil
-            .getLatestBundleID(cluster_2,
-                Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
-        String oldBundleID_cluster3 = InstanceUtil
-            .getLatestBundleID(cluster_3,
-                Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
-        List<String> nominalTimes_cluster3 = OozieUtil.getActionsNominalTime
-            (cluster_3, oldBundleID_cluster3,
-                EntityType.PROCESS);
-
+        String oldBundleID_cluster1 = InstanceUtil.getLatestBundleID(cluster1,
+            Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
+        List<String> nominalTimes_cluster1 = OozieUtil.getActionsNominalTime(cluster1,
+            oldBundleID_cluster1, EntityType.PROCESS);
+        String oldBundleID_cluster2 = InstanceUtil.getLatestBundleID(cluster2,
+            Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
+        String oldBundleID_cluster3 = InstanceUtil.getLatestBundleID(cluster3,
+            Util.readEntityName(processBundle.getProcessData()), EntityType.PROCESS);
+        List<String> nominalTimes_cluster3 = OozieUtil.getActionsNominalTime(cluster3,
+            oldBundleID_cluster3, EntityType.PROCESS);
 
         //update process
         String updateTime = TimeUtil.addMinsToTime(endTime_cluster1, 3);
@@ -641,44 +540,37 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         AssertUtil.assertSucceeded(r);
 
         //check for new bundle to be created
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleID_cluster1, nominalTimes_cluster1,
-            processBundle.getProcessData(), true, false);
-        OozieUtil.verifyNewBundleCreation(cluster_3, oldBundleID_cluster3,
-            nominalTimes_cluster3,
-            processBundle.getProcessData(), true, false);
-        OozieUtil.verifyNewBundleCreation(cluster_2, oldBundleID_cluster2,
-            nominalTimes_cluster3,
-            processBundle.getProcessData(), true, false);
-
-        //wait till new coord are running on Cluster1
-        InstanceUtil.waitTillInstancesAreCreated(cluster_1, processBundle.getProcessData(), 1);
-        OozieUtil.verifyNewBundleCreation(cluster_1, oldBundleID_cluster1, nominalTimes_cluster1,
-            processBundle.getProcessData(), true, true);
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID_cluster1,
+            nominalTimes_cluster1, processBundle.getProcessData(), true, false);
+        OozieUtil.verifyNewBundleCreation(cluster3, oldBundleID_cluster3,
+            nominalTimes_cluster3, processBundle.getProcessData(), true, false);
+        OozieUtil.verifyNewBundleCreation(cluster2, oldBundleID_cluster2,
+            nominalTimes_cluster3, processBundle.getProcessData(), true, false);
+
+        //wait till new coord are running on cluster1
+        InstanceUtil.waitTillInstancesAreCreated(cluster1, processBundle.getProcessData(), 1);
+        OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID_cluster1,
+            nominalTimes_cluster1, processBundle.getProcessData(), true, true);
 
         //verify
-        String coordStartTime_cluster3 = OozieUtil.getCoordStartTime(cluster_3,
+        String coordStartTime_cluster3 = OozieUtil.getCoordStartTime(cluster3,
             processBundle.getProcessData(), 1);
-        String coordStartTime_cluster2 = OozieUtil.getCoordStartTime(cluster_2,
+        String coordStartTime_cluster2 = OozieUtil.getCoordStartTime(cluster2,
             processBundle.getProcessData(), 1);
 
-        if (!(TimeUtil.oozieDateToDate(coordStartTime_cluster3).isAfter
-            (TimeUtil.oozieDateToDate(updateTime)) || TimeUtil
-            .oozieDateToDate(coordStartTime_cluster3).isEqual
-                (TimeUtil.oozieDateToDate(updateTime))))
-            Assert.assertTrue(false, "new coord start time is not correct");
-
-        if (TimeUtil.oozieDateToDate(coordStartTime_cluster2).isEqual
-            (TimeUtil.oozieDateToDate(updateTime)))
-            Assert.assertTrue(false, "new coord start time is not correct");
-
+        DateTime updateTimeOozie = TimeUtil.oozieDateToDate(updateTime);
+        Assert.assertTrue(TimeUtil.oozieDateToDate(coordStartTime_cluster3).isAfter(updateTimeOozie)
+            || TimeUtil.oozieDateToDate(coordStartTime_cluster3).isEqual(updateTimeOozie),
+            "new coord start time is not correct");
+        Assert.assertFalse(
+            TimeUtil.oozieDateToDate(coordStartTime_cluster2).isEqual(updateTimeOozie),
+            "new coord start time is not correct");
         TimeUtil.sleepTill(updateTime);
-
-        InstanceUtil.waitTillInstancesAreCreated(cluster_3, processBundle.getProcessData(), 1);
+        InstanceUtil.waitTillInstancesAreCreated(cluster3, processBundle.getProcessData(), 1);
 
         //verify that no instance are missing
-        OozieUtil.verifyNewBundleCreation(cluster_3, oldBundleID_cluster3,
-            nominalTimes_cluster3,
-            processBundle.getProcessData(), true, true);
+        OozieUtil.verifyNewBundleCreation(cluster3, oldBundleID_cluster3,
+            nominalTimes_cluster3, processBundle.getProcessData(), true, true);
     }
 
     private String submitAndScheduleFeed(Bundle b)
@@ -700,11 +592,9 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         r = prism.getFeedHelper().submitAndSchedule(Util.URLS
             .SUBMIT_AND_SCHEDULE_URL, feed);
         AssertUtil.assertSucceeded(r);
-
         return feed;
     }
 
-
     private String getMultiClusterFeed(String startTimeCluster_source,
                                        String startTimeCluster_target)
         throws IOException, URISyntaxException, AuthenticationException {
@@ -713,25 +603,20 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         //create desired feed
         String feed = bundles[0].getDataSets().get(0);
 
-        //cluster_1 is target, cluster_2 is source and cluster_3 is neutral
-
+        //cluster1 is target, cluster2 is source and cluster3 is neutral
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2010-01-01T00:00Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE), null,
             ClusterType.SOURCE, null);
-
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity(startTimeCluster_source, "2099-10-01T12:10Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
             Util.readEntityName(bundles[2].getClusters().get(0)), null, null);
-
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity(startTimeCluster_target, "2099-10-01T12:25Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
             Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.TARGET,
-            null,
-            testDataDir + dateTemplate);
-
+            null, testDataDir + dateTemplate);
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity(startTimeCluster_source, "2099-01-01T00:00Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
@@ -741,7 +626,7 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         //submit clusters
         Bundle.submitCluster(bundles[0], bundles[1], bundles[2]);
 
-        //create test data on cluster_2
+        //create test data on cluster2
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startTimeCluster_source,
             TimeUtil.getTimeWrtSystemTime(60), 1);
         HadoopUtil.flattenAndPutDataInFolder(cluster2FS, OSUtil.SINGLE_FILE,


[24/41] git commit: FALCON-652 EntityUtils tests are failing. Contributed by Ajay Yadav

Posted by ra...@apache.org.
FALCON-652 EntityUtils tests are failing. Contributed by Ajay Yadav


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

Branch: refs/heads/FALCON-585
Commit: 14b9add5f4af346e9cd517b6360f61fda52005cb
Parents: 9f4a6d2
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Wed Sep 3 16:12:12 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Wed Sep 3 16:12:12 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                                  | 2 ++
 .../test/java/org/apache/falcon/entity/EntityUtilTest.java   | 8 +++++---
 2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/14b9add5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index db2dd04..2086f5b 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -77,6 +77,8 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-652 EntityUtils tests are failing (Ajay Yadav via Venkatesh Seetharam)
+
    FALCON-650 Instance list APIs occassionally fail when orderBy set to
    starttime or endtime (Balu Vellanki via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/14b9add5/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
index 1613a66..50ee649 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
@@ -224,9 +224,11 @@ public class EntityUtilTest extends AbstractTestBase {
 
         process.getClusters().getClusters().add(cluster);
 
+        Date expectedStartDate = new SimpleDateFormat("yyyy-MM-dd z").parse("2011-11-02 UTC");
+        Date expectedEndDate = new SimpleDateFormat("yyyy-MM-dd z").parse("2091-12-30 UTC");
+
         Pair<Date, Date> startEndDates = EntityUtil.getEntityStartEndDates(process);
-        Assert.assertEquals(startEndDates.second.toString(), "Sat Dec 29 16:00:00 PST 2091");
-        Assert.assertEquals(startEndDates.first.toString(), "Tue Nov 01 17:00:00 PDT 2011");
+        Assert.assertEquals(startEndDates.first, expectedStartDate);
+        Assert.assertEquals(startEndDates.second, expectedEndDate);
     }
-
 }


[10/41] git commit: FALCON-474 Add Bulk APIs to drive the dashboard needs. Contributed by Balu Vellanki

Posted by ra...@apache.org.
FALCON-474 Add Bulk APIs to drive the dashboard needs. Contributed by Balu Vellanki


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

Branch: refs/heads/FALCON-585
Commit: 305feb0b4deb9ba949631ccd5963772848b4981b
Parents: e21ec93
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Thu Aug 28 11:13:11 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Thu Aug 28 11:13:11 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../java/org/apache/falcon/cli/FalconCLI.java   |  40 +++-
 .../org/apache/falcon/client/FalconClient.java  |  78 ++++++-
 .../org/apache/falcon/resource/EntityList.java  |   2 +-
 .../falcon/resource/EntitySummaryResult.java    | 220 +++++++++++++++++++
 .../org/apache/falcon/entity/EntityUtil.java    |  47 ++++
 docs/src/site/twiki/FalconCLI.twiki             |  13 ++
 docs/src/site/twiki/restapi/EntityList.twiki    |   2 +-
 docs/src/site/twiki/restapi/EntitySummary.twiki |  72 ++++++
 docs/src/site/twiki/restapi/ResourceList.twiki  |   1 +
 .../falcon/resource/AbstractEntityManager.java  | 131 ++++++-----
 .../resource/AbstractInstanceManager.java       |   2 +-
 .../AbstractSchedulableEntityManager.java       | 107 ++++++++-
 .../proxy/SchedulableEntityManagerProxy.java    |  24 ++
 .../falcon/resource/EntityManagerTest.java      |  30 ++-
 .../resource/SchedulableEntityManager.java      |  23 ++
 .../java/org/apache/falcon/cli/FalconCLIIT.java |  25 +++
 .../falcon/resource/EntityManagerJerseyIT.java  |  13 ++
 .../org/apache/falcon/resource/TestContext.java |   5 +-
 19 files changed, 758 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8bd1ff9..a358be4 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,9 @@ Trunk (Unreleased)
    FALCON-263 API to get workflow parameters. (pavan kumar kolamuri via Shwetha GS)
 
   IMPROVEMENTS
+   FALCON-474 Add Bulk APIs to drive the dashboard needs (Balu Vellanki via
+   Venkatesh Seetharam)
+
    FALCON-166 Instance status start and end dates are rigid and inconvenient
    (Balu Vellanki via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
index cb46d46..f5b30f0 100644
--- a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
+++ b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
@@ -65,6 +65,7 @@ public class FalconCLI {
     public static final String ENTITY_CMD = "entity";
     public static final String ENTITY_TYPE_OPT = "type";
     public static final String COLO_OPT = "colo";
+    public static final String CLUSTER_OPT = "cluster";
     public static final String ENTITY_NAME_OPT = "name";
     public static final String FILE_PATH_OPT = "file";
     public static final String SUBMIT_OPT = "submit";
@@ -87,6 +88,7 @@ public class FalconCLI {
     public static final String ORDER_BY_OPT = "orderBy";
     public static final String OFFSET_OPT = "offset";
     public static final String NUM_RESULTS_OPT = "numResults";
+    public static final String NUM_INSTANCES_OPT = "numInstances";
 
     public static final String INSTANCE_CMD = "instance";
     public static final String START_OPT = "start";
@@ -339,6 +341,9 @@ public class FalconCLI {
         String entityName = commandLine.getOptionValue(ENTITY_NAME_OPT);
         String filePath = commandLine.getOptionValue(FILE_PATH_OPT);
         String colo = commandLine.getOptionValue(COLO_OPT);
+        String cluster = commandLine.getOptionValue(CLUSTER_OPT);
+        String start = commandLine.getOptionValue(START_OPT);
+        String end = commandLine.getOptionValue(END_OPT);
         String time = commandLine.getOptionValue(EFFECTIVE_OPT);
         String orderBy = commandLine.getOptionValue(ORDER_BY_OPT);
         String filterBy = commandLine.getOptionValue(FILTER_BY_OPT);
@@ -347,7 +352,9 @@ public class FalconCLI {
         Integer offset = parseIntegerInput(commandLine.getOptionValue(OFFSET_OPT), 0, "offset");
         Integer numResults = parseIntegerInput(commandLine.getOptionValue(NUM_RESULTS_OPT),
                 FalconClient.DEFAULT_NUM_RESULTS, "numResults");
+        Integer numInstances = parseIntegerInput(commandLine.getOptionValue(NUM_INSTANCES_OPT), 7, "numInstances");
         validateEntityType(entityType);
+        String entityAction = "entity";
 
         if (optionsList.contains(SUBMIT_OPT)) {
             validateFilePath(filePath);
@@ -398,11 +405,18 @@ public class FalconCLI {
         } else if (optionsList.contains(LIST_OPT)) {
             validateColo(optionsList);
             validateEntityFields(fields);
-            validateOrderBy(orderBy, "entity");
-            validateFilterBy(filterBy, "entity");
+            validateOrderBy(orderBy, entityAction);
+            validateFilterBy(filterBy, entityAction);
             EntityList entityList = client.getEntityList(entityType, fields, filterBy,
                     filterTags, orderBy, offset, numResults);
             result = entityList != null ? entityList.toString() : "No entity of type (" + entityType + ") found.";
+        }  else if (optionsList.contains(SUMMARY_OPT)) {
+            validateCluster(cluster);
+            validateEntityFields(fields);
+            validateFilterBy(filterBy, entityAction);
+            validateOrderBy(orderBy, entityAction);
+            result = client.getEntitySummary(entityType, cluster, start, end, fields, filterBy, filterTags,
+                    orderBy, offset, numResults, numInstances);
         } else if (optionsList.contains(HELP_CMD)) {
             OUT.get().println("Falcon Help");
         } else {
@@ -411,6 +425,12 @@ public class FalconCLI {
         OUT.get().println(result);
     }
 
+    private void validateCluster(String cluster) throws FalconCLIException {
+        if (StringUtils.isEmpty(cluster)) {
+            throw new FalconCLIException("Missing argument: cluster");
+        }
+    }
+
     private String getColo(String colo) throws FalconCLIException, IOException {
         if (colo == null) {
             Properties prop = getClientProperties();
@@ -565,6 +585,8 @@ public class FalconCLI {
                 "Gets the dependencies of entity");
         Option list = new Option(LIST_OPT, false,
                 "List entities registerd for a type");
+        Option entitySummary = new Option(SUMMARY_OPT, false,
+                "Get summary of instances for list of entities");
 
         OptionGroup group = new OptionGroup();
         group.addOption(submit);
@@ -579,6 +601,7 @@ public class FalconCLI {
         group.addOption(definition);
         group.addOption(dependency);
         group.addOption(list);
+        group.addOption(entitySummary);
 
         Option url = new Option(URL_OPTION, true, "Falcon URL");
         Option entityType = new Option(ENTITY_TYPE_OPT, true,
@@ -588,8 +611,10 @@ public class FalconCLI {
                 "Path to entity xml file");
         Option entityName = new Option(ENTITY_NAME_OPT, true,
                 "Entity type, can be cluster, feed or process xml");
-        Option colo = new Option(COLO_OPT, true,
-                "Colo name");
+        Option start = new Option(START_OPT, true, "Start time is optional for summary");
+        Option end = new Option(END_OPT, true, "End time is optional for summary");
+        Option colo = new Option(COLO_OPT, true, "Colo name");
+        Option cluster = new Option(CLUSTER_OPT, true, "Cluster name");
         colo.setRequired(false);
         Option effective = new Option(EFFECTIVE_OPT, true, "Effective time for update");
         Option fields = new Option(FIELDS_OPT, true, "Entity fields to show for a request");
@@ -602,6 +627,8 @@ public class FalconCLI {
                 "Start returning entities from this offset");
         Option numResults = new Option(NUM_RESULTS_OPT, true,
                 "Number of results to return per request");
+        Option numInstances = new Option(NUM_INSTANCES_OPT, true,
+                "Number of instances to return per entity summary request");
 
         entityOptions.addOption(url);
         entityOptions.addOptionGroup(group);
@@ -609,6 +636,9 @@ public class FalconCLI {
         entityOptions.addOption(entityName);
         entityOptions.addOption(filePath);
         entityOptions.addOption(colo);
+        entityOptions.addOption(cluster);
+        entityOptions.addOption(start);
+        entityOptions.addOption(end);
         entityOptions.addOption(effective);
         entityOptions.addOption(fields);
         entityOptions.addOption(filterBy);
@@ -616,6 +646,7 @@ public class FalconCLI {
         entityOptions.addOption(orderBy);
         entityOptions.addOption(offset);
         entityOptions.addOption(numResults);
+        entityOptions.addOption(numInstances);
 
         return entityOptions;
     }
@@ -672,7 +703,6 @@ public class FalconCLI {
                 false,
                 "Displays the workflow parameters for a given instance of specified nominal time");
 
-
         OptionGroup group = new OptionGroup();
         group.addOption(running);
         group.addOption(list);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index 5e9543c..619955b 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -31,6 +31,7 @@ import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.EntityList;
+import org.apache.falcon.resource.EntitySummaryResult;
 import org.apache.falcon.resource.InstancesResult;
 import org.apache.falcon.resource.InstancesSummaryResult;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
@@ -180,6 +181,7 @@ public class FalconClient {
         STATUS("api/entities/status/", HttpMethod.GET, MediaType.TEXT_XML),
         DEFINITION("api/entities/definition/", HttpMethod.GET, MediaType.TEXT_XML),
         LIST("api/entities/list/", HttpMethod.GET, MediaType.TEXT_XML),
+        SUMMARY("api/entities/summary", HttpMethod.GET, MediaType.APPLICATION_JSON),
         DEPENDENCY("api/entities/dependencies/", HttpMethod.GET, MediaType.TEXT_XML);
 
         private String path;
@@ -333,13 +335,22 @@ public class FalconClient {
         return sendDependencyRequest(Entities.DEPENDENCY, entityType, entityName);
     }
 
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
+
     public EntityList getEntityList(String entityType, String fields, String filterBy, String filterTags,
                                     String orderBy, Integer offset, Integer numResults) throws FalconCLIException {
         return sendListRequest(Entities.LIST, entityType, fields, filterBy,
                 filterTags, orderBy, offset, numResults);
     }
 
-    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
+    public String getEntitySummary(String entityType, String cluster, String start, String end,
+                                   String fields, String filterBy, String filterTags,
+                                   String orderBy, Integer offset, Integer numResults, Integer numInstances)
+        throws FalconCLIException {
+        return sendEntitySummaryRequest(Entities.SUMMARY, entityType, cluster, start, end, fields, filterBy, filterTags,
+                orderBy, offset, numResults, numInstances);
+    }
+
     public String getRunningInstances(String type, String entity, String colo, List<LifeCycle> lifeCycles,
                                       String filterBy, String orderBy, Integer offset, Integer numResults)
         throws FalconCLIException {
@@ -445,7 +456,6 @@ public class FalconClient {
         return sendInstanceRequest(Instances.PARAMS, type, entity,
                 start, null, null, null, colo, lifeCycles);
     }
-    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
 
     public String getThreadDump() throws FalconCLIException {
         return sendAdminRequest(AdminOperations.STACK);
@@ -523,6 +533,51 @@ public class FalconClient {
         return parseAPIResult(clientResponse);
     }
 
+    private String sendEntitySummaryRequest(Entities entities, String entityType, String cluster,
+                                            String start, String end,
+                                            String fields, String filterBy, String filterTags,
+                                            String orderBy, Integer offset, Integer numResults,
+                                            Integer numInstances) throws FalconCLIException {
+        WebResource resource;
+        if (StringUtils.isEmpty(cluster)) {
+            resource = service.path(entities.path).path(entityType);
+        } else {
+            resource = service.path(entities.path).path(entityType).path(cluster);
+        }
+
+        if (!StringUtils.isEmpty(fields)) {
+            resource = resource.queryParam("fields", fields);
+        }
+        if (!StringUtils.isEmpty(filterTags)) {
+            resource = resource.queryParam("tags", filterTags);
+        }
+        if (!StringUtils.isEmpty(filterBy)) {
+            resource = resource.queryParam("filterBy", filterBy);
+        }
+        if (!StringUtils.isEmpty(orderBy)) {
+            resource = resource.queryParam("orderBy", orderBy);
+        }
+        if (!StringUtils.isEmpty(start)) {
+            resource = resource.queryParam("start", start);
+        }
+        if (!StringUtils.isEmpty(end)) {
+            resource = resource.queryParam("end", end);
+        }
+
+        resource = resource.queryParam("offset", offset.toString());
+        resource = resource.queryParam("numResults", numResults.toString());
+        resource = resource.queryParam("numInstances", numInstances.toString());
+
+        ClientResponse clientResponse = resource
+                .header("Cookie", AUTH_COOKIE_EQ + authenticationToken)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class);
+
+        checkIfSuccessful(clientResponse);
+        return parseProcessEntitySummaryResult(clientResponse);
+    }
+    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
+
     private String sendDefinitionRequest(Entities entities, String entityType,
                                          String entityName) throws FalconCLIException {
 
@@ -724,6 +779,25 @@ public class FalconClient {
         return clientResponse.getEntity(String.class);
     }
 
+    private String parseProcessEntitySummaryResult(ClientResponse clientResponse) {
+        EntitySummaryResult result = clientResponse.getEntity(EntitySummaryResult.class);
+        StringBuilder sb = new StringBuilder();
+        String toAppend;
+        sb.append("Consolidated Status: ").append(result.getStatus()).append("\n");
+        sb.append("\nEntity Summary Result :\n");
+        if (result.getEntitySummaries() != null) {
+            for (EntitySummaryResult.EntitySummary entitySummary : result.getEntitySummaries()) {
+
+                toAppend = entitySummary.toString();
+                sb.append(toAppend).append("\n");
+            }
+        }
+        sb.append("\nAdditional Information:\n");
+        sb.append("Response: ").append(result.getMessage());
+        sb.append("Request Id: ").append(result.getRequestId());
+        return sb.toString();
+    }
+
     private String summarizeProcessInstanceResult(ClientResponse clientResponse) {
         InstancesSummaryResult result = clientResponse
                 .getEntity(InstancesSummaryResult.class);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/client/src/main/java/org/apache/falcon/resource/EntityList.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/EntityList.java b/client/src/main/java/org/apache/falcon/resource/EntityList.java
index f67b84b..243c119 100644
--- a/client/src/main/java/org/apache/falcon/resource/EntityList.java
+++ b/client/src/main/java/org/apache/falcon/resource/EntityList.java
@@ -51,7 +51,7 @@ public class EntityList {
      * Filter by these Fields is supported by RestAPI.
      */
     public static enum EntityFilterByFields {
-        TYPE, NAME, STATUS, PIPELINES
+        TYPE, NAME, STATUS, PIPELINES, CLUSTER
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/client/src/main/java/org/apache/falcon/resource/EntitySummaryResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/EntitySummaryResult.java b/client/src/main/java/org/apache/falcon/resource/EntitySummaryResult.java
new file mode 100644
index 0000000..4a885ec
--- /dev/null
+++ b/client/src/main/java/org/apache/falcon/resource/EntitySummaryResult.java
@@ -0,0 +1,220 @@
+/**
+ * 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.resource;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.Arrays;
+import java.util.Date;
+
+/**
+ * Pojo for JAXB marshalling / unmarshalling.
+ */
+//SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
+@XmlRootElement
+@edu.umd.cs.findbugs.annotations.SuppressWarnings({"EI_EXPOSE_REP", "EI_EXPOSE_REP2"})
+public class EntitySummaryResult extends APIResult {
+
+    /**
+     * Workflow status as being set in result object.
+     */
+    public static enum WorkflowStatus {
+        WAITING, RUNNING, SUSPENDED, KILLED, FAILED, SUCCEEDED, ERROR
+    }
+
+    @XmlElement
+    private EntitySummary[] entitySummaries;
+
+    //For JAXB
+    public EntitySummaryResult() {
+        super();
+    }
+
+    public EntitySummaryResult(String message, EntitySummary[] entitySummaries) {
+        this(Status.SUCCEEDED, message, entitySummaries);
+    }
+
+    public EntitySummaryResult(Status status, String message, EntitySummary[] entitySummaries) {
+        super(status, message);
+        this.entitySummaries = entitySummaries;
+    }
+
+    public EntitySummaryResult(Status status, String message) {
+        super(status, message);
+    }
+
+    public EntitySummary[] getEntitySummaries() {
+        return this.entitySummaries;
+    }
+
+    public void setEntitySummaries(EntitySummary[] entitySummaries) {
+        this.entitySummaries = entitySummaries;
+    }
+
+    /**
+     * A single entity object inside entity summary result.
+     */
+    @XmlRootElement(name = "entitySummary")
+    public static class EntitySummary {
+
+        @XmlElement
+        public String type;
+        @XmlElement
+        public String name;
+        @XmlElement
+        public String status;
+        @XmlElement
+        public String[] tags;
+        @XmlElement
+        public String[] pipelines;
+        @XmlElement
+        public Instance[] instances;
+
+        public EntitySummary() {
+        }
+
+        public EntitySummary(String entityName, String entityType) {
+            this.name = entityName;
+            this.type = entityType;
+        }
+
+        public EntitySummary(String name, String type, String status,
+                             String[] tags, String[] pipelines,
+                             Instance[] instances) {
+            this.name = name;
+            this.type = type;
+            this.status = status;
+            this.pipelines = pipelines;
+            this.tags = tags;
+            this.instances = instances;
+        }
+
+        public String getName() {
+            return this.name;
+        }
+
+        public String getType() {
+            return this.type;
+        }
+
+        public String getStatus() {
+            return this.status;
+        }
+
+        public String[] getTags() {
+            return this.tags;
+        }
+
+        public String[] getPipelines() {
+            return this.pipelines;
+        }
+
+        public Instance[] getInstances() {
+            return this.instances;
+        }
+
+        @Override
+        public String toString() {
+            return "{Entity: " + (this.name == null ? "" : this.name)
+                    + ", Type: " + (this.type == null ? "" : this.type)
+                    + ", Status: " + (this.status == null ? "" : this.status)
+                    + ", Tags: " + (this.tags == null ? "[]" : Arrays.toString(this.tags))
+                    + ", Pipelines: " + (this.pipelines == null ? "[]" : Arrays.toString(this.pipelines))
+                    + ", InstanceSummary: " + (this.instances == null ? "[]" : Arrays.toString(this.instances))
+                    +"}";
+        }
+    }
+
+    /**
+     * A single instance object inside instance result.
+     */
+    @XmlRootElement(name = "instances")
+    public static class Instance {
+        @XmlElement
+        public String instance;
+
+        @XmlElement
+        public WorkflowStatus status;
+
+        @XmlElement
+        public String logFile;
+
+        @XmlElement
+        public String cluster;
+
+        @XmlElement
+        public String sourceCluster;
+
+        @XmlElement
+        public Date startTime;
+
+        @XmlElement
+        public Date endTime;
+
+        public Instance() {
+        }
+
+        public Instance(String cluster, String instance, WorkflowStatus status) {
+            this.cluster = cluster;
+            this.instance = instance;
+            this.status = status;
+        }
+
+        public String getInstance() {
+            return instance;
+        }
+
+        public WorkflowStatus getStatus() {
+            return status;
+        }
+
+        public String getLogFile() {
+            return logFile;
+        }
+
+        public String getCluster() {
+            return cluster;
+        }
+
+        public String getSourceCluster() {
+            return sourceCluster;
+        }
+
+        public Date getStartTime() {
+            return startTime;
+        }
+
+        public Date getEndTime() {
+            return endTime;
+        }
+
+        @Override
+        public String toString() {
+            return "{instance: " + (this.instance == null ? "" : this.instance)
+                    + ", status: " + (this.status == null ? "" : this.status)
+                    + (this.logFile == null ? "" : ", log: " + this.logFile)
+                    + (this.sourceCluster == null ? "" : ", source-cluster: " + this.sourceCluster)
+                    + (this.cluster == null ? "" : ", cluster: " + this.cluster)
+                    + (this.startTime == null ? "" : ", startTime: " + this.startTime)
+                    + (this.endTime == null ? "" : ", endTime: " + this.endTime)
+                    + "}";
+        }
+    }
+}
+//RESUME CHECKSTYLE CHECK VisibilityModifierCheck

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
index e75f28e..8f258fb 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -21,6 +21,7 @@ package org.apache.falcon.entity;
 import org.apache.commons.beanutils.PropertyUtils;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Pair;
 import org.apache.falcon.Tag;
@@ -688,6 +689,52 @@ public final class EntityUtil {
         return Storage.TYPE.TABLE == storageType;
     }
 
+    public static List<String> getTags(Entity entity) {
+        String rawTags = null;
+
+        switch (entity.getEntityType()) {
+        case PROCESS:
+            rawTags = ((Process) entity).getTags();
+            break;
+
+        case FEED:
+            rawTags = ((Feed) entity).getTags();
+            break;
+
+        case CLUSTER:
+            rawTags = ((Cluster) entity).getTags();
+            break;
+
+        default:
+            break;
+        }
+
+        List<String> tags = new ArrayList<String>();
+        if (!StringUtils.isEmpty(rawTags)) {
+            for(String tag : rawTags.split(",")) {
+                tags.add(tag.trim());
+            }
+        }
+
+        return tags;
+    }
+
+    public static List<String> getPipelines(Entity entity) {
+        List<String> pipelines = new ArrayList<String>();
+
+        if (entity.getEntityType().equals(EntityType.PROCESS)) {
+            Process process = (Process) entity;
+            String pipelineString = process.getPipelines();
+            if (pipelineString != null) {
+                for (String pipeline : pipelineString.split(",")) {
+                    pipelines.add(pipeline.trim());
+                }
+            }
+        } // else : Pipelines are only set for Process entities
+
+        return pipelines;
+    }
+
     public static Pair<Date, Date> getEntityStartEndDates(Entity entityObject) {
         Set<String> clusters = EntityUtil.getClustersDefined(entityObject);
         Pair<Date, String> clusterMinStartDate = null;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/docs/src/site/twiki/FalconCLI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/FalconCLI.twiki b/docs/src/site/twiki/FalconCLI.twiki
index 77a306e..4ea80c1 100644
--- a/docs/src/site/twiki/FalconCLI.twiki
+++ b/docs/src/site/twiki/FalconCLI.twiki
@@ -56,6 +56,19 @@ Optional Args : -fields <<field1,field2>> -filterBy <<field1:value1,field2:value
 
 <a href="./Restapi/EntityList.html">Optional params described here.</a>
 
+---+++Summary
+
+Summary of entities of a particular type and a cluster will be listed. Entity summary has N most recent instances of entity.
+
+Usage:
+$FALCON_HOME/bin/falcon entity -type [cluster|feed|process] -summary
+
+Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'" -fields <<field1,field2>>
+-filterBy <<field1:value1,field2:value2>> -tags <<tagkey=tagvalue,tagkey=tagvalue>>
+-orderBy <<field>> -offset 0 -numResults 10 -numInstances 7
+
+<a href="./Restapi/EntitySummary.html">Optional params described here.</a>
+
 ---+++Update
 
 Update operation allows an already submitted/scheduled entity to be updated. Cluster update is currently

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/docs/src/site/twiki/restapi/EntityList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/EntityList.twiki b/docs/src/site/twiki/restapi/EntityList.twiki
index 0697561..353007c 100644
--- a/docs/src/site/twiki/restapi/EntityList.twiki
+++ b/docs/src/site/twiki/restapi/EntityList.twiki
@@ -12,7 +12,7 @@ Get list of the entities.
    * fields <optional param> Fields of entity that the user wants to view, separated by commas.
       * Valid options are STATUS, TAGS, PIPELINES.
    * filterBy <optional param> Filter results by list of field:value pairs. Example: filterBy=STATUS:RUNNING,PIPELINES:clickLogs
-      * Supported filter fields are NAME, STATUS, PIPELINES.
+      * Supported filter fields are NAME, STATUS, PIPELINES, CLUSTER.
       * Query will do an AND among filterBy fields.
    * tags <optional param> Return list of entities that have specified tags, separated by a comma. Query will do AND on tag values.
       * Example: tags=consumer=consumer@xyz.com,owner=producer@xyz.com

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/docs/src/site/twiki/restapi/EntitySummary.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/EntitySummary.twiki b/docs/src/site/twiki/restapi/EntitySummary.twiki
new file mode 100644
index 0000000..6e6ddf4
--- /dev/null
+++ b/docs/src/site/twiki/restapi/EntitySummary.twiki
@@ -0,0 +1,72 @@
+---++  GET /api/entities/summary/:entity-type/:cluster
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Given an EntityType and cluster, get list of entities along with summary of N recent instances of each entity
+
+---++ Parameters
+   * :entity-type Valid options are cluster, feed or process.
+   * :cluster Show entities that belong to this cluster.
+   * start <optional param> Show entity summaries from this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * By default, it is set to (end - 2 days).
+   * end <optional param> Show entity summary up to this date. Date format is yyyy-MM-dd'T'HH:mm'Z'.
+      * Default is set to now.
+   * fields <optional param> Fields of entity that the user wants to view, separated by commas.
+      * Valid options are STATUS, TAGS, PIPELINES.
+   * filterBy <optional param> Filter results by list of field:value pairs. Example: filterBy=STATUS:RUNNING,PIPELINES:clickLogs
+      * Supported filter fields are NAME, STATUS, PIPELINES, CLUSTER.
+      * Query will do an AND among filterBy fields.
+   * tags <optional param> Return list of entities that have specified tags, separated by a comma. Query will do AND on tag values.
+      * Example: tags=consumer=consumer@xyz.com,owner=producer@xyz.com
+   * orderBy <optional param> Field by which results should be ordered.
+      * Supports ordering by "name".
+   * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
+   * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
+   * numInstances <optional param> Number of recent instances to show per entity. Only integers > 0 are valid, Default is 7.
+
+---++ Results
+Show entities along with summary of N instances for each entity.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/entities/summary/feed/primary-cluster?filterBy=STATUS:RUNNING&fields=status&tags=consumer=consumer@xyz.com&orderBy=name&offset=0&numResults=1&numInstances=2
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "entitySummary": [
+        {
+            "name"  : "SampleOutput",
+            "type"  : "feed",
+            "status": "RUNNING",
+            "instances": [
+            {
+                "details": "",
+                "endTime": "2013-10-21T14:40:26-07:00",
+                "startTime": "2013-10-21T14:39:56-07:00",
+                "cluster": "primary-cluster",
+                "logFile": "http:\/\/localhost:11000\/oozie?job=0000070-131021115933395-oozie-rgau-W",
+                "status": "RUNNING",
+                "instance": "2012-04-03T07:00Z"
+            },
+            {
+                "details": "",
+                "endTime": "2013-10-21T14:42:27-07:00",
+                "startTime": "2013-10-21T14:41:57-07:00",
+                "cluster": "primary-cluster",
+                "logFile": "http:\/\/localhost:11000\/oozie?job=0000070-131021115933397-oozie-rgau-W",
+                "status": "RUNNING",
+                "instance": "2012-04-03T08:00Z"
+            },
+            ]
+        }
+    ]
+    "requestId": "default\/e15bb378-d09f-4911-9df2-5334a45153d2\n",
+    "message": "default\/STATUS\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/docs/src/site/twiki/restapi/ResourceList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/ResourceList.twiki b/docs/src/site/twiki/restapi/ResourceList.twiki
index 9284810..d9cb3cb 100644
--- a/docs/src/site/twiki/restapi/ResourceList.twiki
+++ b/docs/src/site/twiki/restapi/ResourceList.twiki
@@ -49,6 +49,7 @@ See also: [[../Security.twiki][Security in Falcon]]
 | GET         | [[EntityStatus][api/entities/status/:entity-type/:entity-name]]             | Get the status of the entity       |
 | GET         | [[EntityDefinition][api/entities/definition/:entity-type/:entity-name]]     | Get the definition of the entity   |
 | GET         | [[EntityList][api/entities/list/:entity-type]]                              | Get the list of entities           |
+| GET         | [[EntitySummary][api/entities/summary/:entity-type/:cluster]]               | Get instance summary of all entities |
 | GET         | [[EntityDependencies][api/entities/dependencies/:entity-type/:entity-name]] | Get the dependencies of the entity |
 
 ---++ REST Call on Feed and Process Instances

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
index a6d1b29..d12dede 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
@@ -20,6 +20,7 @@ package org.apache.falcon.resource;
 
 import org.apache.commons.beanutils.PropertyUtils;
 import org.apache.commons.lang.ObjectUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.FalconWebException;
@@ -33,8 +34,6 @@ import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.store.EntityAlreadyExistsException;
 import org.apache.falcon.entity.v0.*;
 import org.apache.falcon.entity.v0.cluster.Cluster;
-import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.resource.APIResult.Status;
 import org.apache.falcon.resource.EntityList.EntityElement;
 import org.apache.falcon.security.CurrentUser;
@@ -44,7 +43,6 @@ import org.apache.falcon.util.RuntimeProperties;
 import org.apache.falcon.workflow.WorkflowEngineFactory;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -461,27 +459,46 @@ public abstract class AbstractEntityManager {
         }
     }
 
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     /**
      * Returns the list of entities registered of a given type.
      *
      * @param type           Only return entities of this type
      * @param fieldStr       fields that the query is interested in, separated by comma
      * @param filterBy       filter by a specific field.
-     * @param offset         Pagination offset
-     * @param resultsPerPage Number of results that should be returned starting at the offset
+     * @param filterTags     filter by these tags.
+     * @param orderBy        order result by these fields.
+     * @param offset         Pagination offset.
+     * @param resultsPerPage Number of results that should be returned starting at the offset.
      * @return EntityList
      */
     public EntityList getEntityList(String type, String fieldStr, String filterBy, String filterTags,
                                     String orderBy, Integer offset, Integer resultsPerPage) {
 
         HashSet<String> fields = new HashSet<String>(Arrays.asList(fieldStr.toLowerCase().split(",")));
+        List<Entity> entities;
+        try {
+            entities = getEntities(type, "", "", "", filterBy, filterTags, orderBy, offset, resultsPerPage);
+        } catch (Exception e) {
+            LOG.error("Failed to get entity list", e);
+            throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
+        }
+
+        return entities.size() == 0
+                ? new EntityList(new Entity[]{})
+                : new EntityList(buildEntityElements(fields, entities));
+    }
+
+    protected List<Entity> getEntities(String type, String startDate, String endDate, String cluster,
+                                       String filterBy, String filterTags, String orderBy,
+                                       int offset, int resultsPerPage) throws FalconException {
         final HashMap<String, String> filterByFieldsValues = getFilterByFieldsValues(filterBy);
         final ArrayList<String> filterByTags = getFilterByTags(filterTags);
 
         EntityType entityType = EntityType.valueOf(type.toUpperCase());
         Collection<String> entityNames = configStore.getEntities(entityType);
         if (entityNames == null || entityNames.isEmpty()) {
-            return new EntityList(new Entity[]{});
+            return Collections.emptyList();
         }
 
         ArrayList<Entity> entities = new ArrayList<Entity>();
@@ -497,8 +514,12 @@ public abstract class AbstractEntityManager {
                 throw FalconWebException.newException(e1, Response.Status.BAD_REQUEST);
             }
 
-            List<String> tags = getTags(entity);
-            List<String> pipelines = getPipelines(entity);
+            if (filterEntityByDatesAndCluster(entity, startDate, endDate, cluster)) {
+                continue;
+            }
+
+            List<String> tags = EntityUtil.getTags(entity);
+            List<String> pipelines = EntityUtil.getPipelines(entity);
             String entityStatus = getStatusString(entity);
 
             if (filterEntity(entity, entityStatus,
@@ -512,10 +533,37 @@ public abstract class AbstractEntityManager {
 
         int pageCount = getRequiredNumberOfResults(entities.size(), offset, resultsPerPage);
         if (pageCount == 0) {  // handle pagination
-            return new EntityList(new Entity[]{});
+            return new ArrayList<Entity>();
+        }
+
+        return new ArrayList<Entity>(entities.subList(offset, (offset + pageCount)));
+    }
+    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
+
+    private boolean filterEntityByDatesAndCluster(Entity entity, String startDate, String endDate, String cluster)
+        throws FalconException {
+        if (StringUtils.isEmpty(cluster)) {
+            return false; // no filtering necessary on cluster
+        }
+        Set<String> clusters = EntityUtil.getClustersDefined(entity);
+        if (!clusters.contains(cluster)) {
+            return true; // entity does not have this cluster
+        }
+
+        if (!StringUtils.isEmpty(startDate)) {
+            Date parsedDate = EntityUtil.parseDateUTC(startDate);
+            if (parsedDate.after(EntityUtil.getEndTime(entity, cluster))) {
+                return true;
+            }
+        }
+        if (!StringUtils.isEmpty(endDate)) {
+            Date parseDate = EntityUtil.parseDateUTC(endDate);
+            if (parseDate.before(EntityUtil.getStartTime(entity, cluster))) {
+                return true;
+            }
         }
 
-        return new EntityList(buildEntityElements(offset, fields, entities, pageCount));
+        return false;
     }
 
     protected static HashMap<String, String> getFilterByFieldsValues(String filterBy) {
@@ -547,51 +595,7 @@ public abstract class AbstractEntityManager {
         return filterTagsList;
     }
 
-    private List<String> getTags(Entity entity) {
-        String rawTags = null;
-        switch (entity.getEntityType()) {
-        case PROCESS:
-            rawTags = ((Process) entity).getTags();
-            break;
-
-        case FEED:
-            rawTags = ((Feed) entity).getTags();
-            break;
-
-        case CLUSTER:
-            rawTags = ((Cluster) entity).getTags();
-            break;
-
-        default:
-            break;
-        }
-
-        List<String> tags = new ArrayList<String>();
-        if (!StringUtils.isEmpty(rawTags)) {
-            for(String tag : rawTags.split(",")) {
-                LOG.info("Adding tag - "+ tag);
-                tags.add(tag.trim());
-            }
-        }
-
-        return tags;
-    }
-
-    private List<String> getPipelines(Entity entity) {
-        List<String> pipelines = new ArrayList<String>();
-        if (entity.getEntityType().equals(EntityType.PROCESS)) {
-            Process process = (Process) entity;
-            String pipelineString = process.getPipelines();
-            if (pipelineString != null) {
-                for (String pipeline : pipelineString.split(",")) {
-                    pipelines.add(pipeline.trim());
-                }
-            }
-        }
-        return pipelines;
-    }
-
-    private String getStatusString(Entity entity) {
+    protected String getStatusString(Entity entity) {
         String statusString;
         try {
             statusString = getStatus(entity, entity.getEntityType()).name();
@@ -681,6 +685,12 @@ public abstract class AbstractEntityManager {
                     }
                     break;
 
+                case CLUSTER:
+                    Set<String> clusters = EntityUtil.getClustersDefined(entity);
+                    if (!clusters.contains(filterValue)) {
+                        filterEntity = true;
+                    }
+
                 default:
                     break;
                 }
@@ -751,11 +761,10 @@ public abstract class AbstractEntityManager {
         return retLen;
     }
 
-    private EntityElement[] buildEntityElements(Integer offset, HashSet<String> fields,
-                                                ArrayList<Entity> entities, int pageCount) {
-        EntityElement[] elements = new EntityElement[pageCount];
+    private EntityElement[] buildEntityElements(HashSet<String> fields, List<Entity> entities) {
+        EntityElement[] elements = new EntityElement[entities.size()];
         int elementIndex = 0;
-        for (Entity entity : entities.subList(offset, (offset + pageCount))) {
+        for (Entity entity : entities) {
             elements[elementIndex++] = getEntityElement(entity, fields);
         }
         return elements;
@@ -769,10 +778,10 @@ public abstract class AbstractEntityManager {
             elem.status = getStatusString(entity);
         }
         if (fields.contains("pipelines")) {
-            elem.pipelines = getPipelines(entity);
+            elem.pipelines = EntityUtil.getPipelines(entity);
         }
         if (fields.contains("tags")) {
-            elem.tag = getTags(entity);
+            elem.tag = EntityUtil.getTags(entity);
         }
 
         return elem;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
index 6862b99..1ffe471 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
@@ -46,7 +46,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
 
     private static final long MINUTE_IN_MILLIS = 60000L;
     private static final long HOUR_IN_MILLIS = 3600000L;
-    private static final long DAY_IN_MILLIS = 86400000L;
+    protected static final long DAY_IN_MILLIS = 86400000L;
     private static final long MONTH_IN_MILLIS = 2592000000L;
 
     protected void checkType(String type) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
index f5329ef..6fba6df 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
@@ -18,12 +18,16 @@
 
 package org.apache.falcon.resource;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconWebException;
+import org.apache.falcon.Pair;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
+import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.entity.v0.UnschedulableEntityException;
+import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.monitors.Dimension;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.slf4j.Logger;
@@ -33,12 +37,13 @@ import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
+import java.util.*;
 
 /**
  * REST resource of allowed actions on Schedulable Entities, Only Process and
  * Feed can have schedulable actions.
  */
-public abstract class AbstractSchedulableEntityManager extends AbstractEntityManager {
+public abstract class AbstractSchedulableEntityManager extends AbstractInstanceManager {
 
     private static final Logger LOG = LoggerFactory.getLogger(AbstractSchedulableEntityManager.class);
 
@@ -152,6 +157,106 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
         }
     }
 
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
+    /**
+     * Returns summary of most recent N instances of an entity, filtered by cluster.
+     *
+     * @param type           Only return entities of this type.
+     * @param startDate      For each entity, show instances after startDate.
+     * @param endDate        For each entity, show instances before endDate.
+     * @param cluster        Return entities for specific cluster.
+     * @param fields       fields that the query is interested in, separated by comma
+     * @param filterBy       filter by a specific field.
+     * @param filterTags     filter by these tags.
+     * @param orderBy        order result by these fields.
+     * @param offset         Pagination offset.
+     * @param resultsPerPage Number of results that should be returned starting at the offset.
+     * @param numInstances   Number of instance summaries to show per entity
+     * @return EntitySummaryResult
+     */
+    public EntitySummaryResult getEntitySummary(String type, String cluster, String startDate, String endDate,
+                                                String fields, String filterBy, String filterTags,
+                                                String orderBy, Integer offset,
+                                                Integer resultsPerPage, Integer numInstances) {
+        HashSet<String> fieldSet = new HashSet<String>(Arrays.asList(fields.toLowerCase().split(",")));
+        Pair<Date, Date> startAndEndDates = getStartEndDatesForSummary(startDate, endDate);
+
+        List<Entity> entities;
+        String colo;
+        try {
+            entities = getEntities(type,
+                    SchemaHelper.getDateFormat().format(startAndEndDates.first),
+                    SchemaHelper.getDateFormat().format(startAndEndDates.second),
+                    cluster, filterBy, filterTags, orderBy, offset, resultsPerPage);
+            colo = ((Cluster) configStore.get(EntityType.CLUSTER, cluster)).getColo();
+        } catch (Exception e) {
+            LOG.error("Failed to get entities", e);
+            throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
+        }
+
+        List<EntitySummaryResult.EntitySummary> entitySummaries = new ArrayList<EntitySummaryResult.EntitySummary>();
+        for (Entity entity : entities) {
+            InstancesResult instancesResult = getInstances(entity.getEntityType().name(), entity.getName(),
+                    SchemaHelper.getDateFormat().format(startAndEndDates.first),
+                    SchemaHelper.getDateFormat().format(startAndEndDates.second),
+                    colo, null, "", "", 0, numInstances);
+
+            /* ToDo - Use oozie bulk API after FALCON-591 is implemented
+             *       getBulkInstances(entity, cluster,
+             *      startAndEndDates.first, startAndEndDates.second, colo, "starttime", 0, numInstances);
+             */
+            List<EntitySummaryResult.Instance> entitySummaryInstances =
+                    getElementsFromInstanceResult(instancesResult);
+
+            List<String> pipelines = new ArrayList<String>();
+            List<String> tags = new ArrayList<String>();
+            if (fieldSet.contains("pipelines")) { pipelines = EntityUtil.getPipelines(entity); }
+            if (fieldSet.contains("tags")) { tags = EntityUtil.getTags(entity); }
+
+            EntitySummaryResult.EntitySummary entitySummary =
+                    new EntitySummaryResult.EntitySummary(entity.getName(), entity.getEntityType().toString(),
+                            getStatusString(entity),
+                            tags.toArray(new String[tags.size()]),
+                            pipelines.toArray(new String[pipelines.size()]),
+                            entitySummaryInstances.toArray(
+                                    new EntitySummaryResult.Instance[entitySummaryInstances.size()]));
+            entitySummaries.add(entitySummary);
+        }
+        return new EntitySummaryResult("Entity Summary Result",
+                entitySummaries.toArray(new EntitySummaryResult.EntitySummary[entitySummaries.size()]));
+    }
+    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
+
+    private Pair<Date, Date> getStartEndDatesForSummary(String startDate, String endDate) {
+        Date end = (StringUtils.isEmpty(endDate)) ? new Date() : SchemaHelper.parseDateUTC(endDate);
+
+        long startMillisecs = end.getTime() - (2* DAY_IN_MILLIS); // default - 2 days before end
+        Date start = (StringUtils.isEmpty(startDate))
+                ? new Date(startMillisecs) : SchemaHelper.parseDateUTC(startDate);
+
+        return new Pair<Date, Date>(start, end);
+    }
+
+    private List<EntitySummaryResult.Instance> getElementsFromInstanceResult(InstancesResult instancesResult) {
+        ArrayList<EntitySummaryResult.Instance> elemInstanceList =
+                new ArrayList<EntitySummaryResult.Instance>();
+        InstancesResult.Instance[] instances = instancesResult.getInstances();
+        if (instances != null && instances.length > 0) {
+            for (InstancesResult.Instance rawInstance : instances) {
+                EntitySummaryResult.Instance instance = new EntitySummaryResult.Instance(rawInstance.getCluster(),
+                        rawInstance.getInstance(),
+                        EntitySummaryResult.WorkflowStatus.valueOf(rawInstance.getStatus().toString()));
+                instance.logFile = rawInstance.getLogFile();
+                instance.sourceCluster = rawInstance.sourceCluster;
+                instance.startTime = rawInstance.startTime;
+                instance.endTime = rawInstance.endTime;
+                elemInstanceList.add(instance);
+            }
+        }
+
+        return elemInstanceList;
+    }
+
     private void checkSchedulableEntity(String type) throws UnschedulableEntityException {
         EntityType entityType = EntityType.valueOf(type.toUpperCase());
         if (!entityType.isSchedulable()) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
index f743005..e69e531 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
@@ -30,6 +30,7 @@ import org.apache.falcon.monitors.Monitored;
 import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.AbstractSchedulableEntityManager;
 import org.apache.falcon.resource.EntityList;
+import org.apache.falcon.resource.EntitySummaryResult;
 import org.apache.falcon.resource.channel.Channel;
 import org.apache.falcon.resource.channel.ChannelFactory;
 import org.apache.falcon.util.DeploymentUtil;
@@ -403,6 +404,29 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
         }.execute();
     }
 
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
+    @GET
+    @Path("summary/{type}/{cluster}")
+    @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
+    @Monitored(event = "summary")
+    @Override
+    public EntitySummaryResult getEntitySummary(
+            @Dimension("type") @PathParam("type") final String type,
+            @Dimension("cluster") @PathParam("cluster") final String cluster,
+            @DefaultValue("") @QueryParam("start") String startStr,
+            @DefaultValue("") @QueryParam("end") String endStr,
+            @DefaultValue("") @QueryParam("fields") final String entityFields,
+            @DefaultValue("") @QueryParam("filterBy") final String entityFilter,
+            @DefaultValue("") @QueryParam("tags") final String entityTags,
+            @DefaultValue("") @QueryParam("orderBy") final String entityOrderBy,
+            @DefaultValue("0") @QueryParam("offset") final Integer entityOffset,
+            @DefaultValue("10") @QueryParam("numResults") final Integer numEntities,
+            @DefaultValue("7") @QueryParam("numInstances") final Integer numInstanceResults) {
+        return super.getEntitySummary(type, cluster, startStr, endStr, entityFields, entityFilter, entityTags,
+                entityOrderBy, entityOffset, numEntities, numInstanceResults);
+    }
+    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
+
     private abstract class EntityProxy {
         private String type;
         private String name;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
index 2c0fa25..0b55eb3 100644
--- a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
+++ b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
@@ -21,7 +21,9 @@ import org.apache.falcon.FalconWebException;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.process.ACL;
+import org.apache.falcon.entity.v0.process.Clusters;
 import org.apache.falcon.entity.v0.process.Process;
+import org.apache.falcon.entity.v0.process.Validity;
 import org.apache.falcon.security.CurrentUser;
 import org.apache.falcon.util.StartupProperties;
 import org.mockito.Mock;
@@ -35,6 +37,7 @@ import javax.servlet.ServletInputStream;
 import javax.servlet.http.HttpServletRequest;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.Date;
 
 import static org.mockito.Mockito.when;
 
@@ -48,6 +51,7 @@ public class EntityManagerTest extends AbstractEntityManager {
     private static final String SAMPLE_PROCESS_XML = "/process-version-0.xml";
 
     private static final String SAMPLE_INVALID_PROCESS_XML = "/process-invalid.xml";
+    private static final long DAY_IN_MILLIS = 86400000L;
 
     @BeforeClass
     public void init() {
@@ -199,14 +203,28 @@ public class EntityManagerTest extends AbstractEntityManager {
         acl.setGroup("hdfs");
         acl.setPermission("*");
 
-        Process p = new Process();
-        p.setName(name);
-        p.setACL(acl);
-        p.setPipelines(pipelines);
-        p.setTags(tags);
-        return p;
+        Process process = new Process();
+        process.setName(name);
+        process.setACL(acl);
+        process.setPipelines(pipelines);
+        process.setTags(tags);
+        process.setClusters(buildClusters("cluster" + name));
+        return process;
     }
 
+    private Clusters buildClusters(String name) {
+        Validity validity = new Validity();
+        long startMilliSecs = new Date().getTime() - (2 * DAY_IN_MILLIS);
+        validity.setStart(new Date(startMilliSecs));
+        validity.setEnd(new Date());
+        org.apache.falcon.entity.v0.process.Cluster cluster = new org.apache.falcon.entity.v0.process.Cluster();
+        cluster.setName(name);
+        cluster.setValidity(validity);
+
+        Clusters clusters =  new Clusters();
+        clusters.getClusters().add(cluster);
+        return clusters;
+    }
 
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
index 58a3bd2..c10301b 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
@@ -69,6 +69,29 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
         return super.getEntityList(type, fields, filterBy, tags, orderBy, offset, resultsPerPage);
     }
 
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
+    @GET
+    @Path("summary/{type}/{cluster}")
+    @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
+    @Monitored(event = "summary")
+    @Override
+    public EntitySummaryResult getEntitySummary(
+            @Dimension("type") @PathParam("type") String type,
+            @Dimension("cluster") @PathParam("cluster") String cluster,
+            @DefaultValue("") @QueryParam("start") String startStr,
+            @DefaultValue("") @QueryParam("end") String endStr,
+            @DefaultValue("") @QueryParam("fields") String fields,
+            @DefaultValue("") @QueryParam("filterBy") String entityFilter,
+            @DefaultValue("") @QueryParam("tags")  String entityTags,
+            @DefaultValue("") @QueryParam("orderBy") String entityOrderBy,
+            @DefaultValue("0") @QueryParam("offset") Integer entityOffset,
+            @DefaultValue("10") @QueryParam("numResults") Integer numEntities,
+            @DefaultValue("7") @QueryParam("numInstances") Integer numInstanceResults) {
+        return super.getEntitySummary(type, cluster, startStr, endStr, fields, entityFilter, entityTags,
+                entityOrderBy, entityOffset, numEntities, numInstanceResults);
+    }
+    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
+
     @GET
     @Path("definition/{type}/{entity}")
     @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN, MediaType.APPLICATION_JSON})

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
index 4ac978c..df99c23 100644
--- a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
+++ b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
@@ -285,6 +285,25 @@ public class FalconCLIIT {
                 executeWithURL("entity -status -type process -name "
                         + overlay.get("processName")));
 
+        Assert.assertEquals(0,
+                executeWithURL("entity -summary -type feed -cluster "+ overlay.get("cluster")
+                        + " -fields status,tags -start " + START_INSTANCE
+                        + " -filterBy TYPE:FEED -orderBy name "
+                        + " -offset 0 -numResults 1 -numInstances 5"));
+        Assert.assertEquals(0,
+                executeWithURL("entity -summary -type process -fields status,pipelines"
+                        + " -cluster " + overlay.get("cluster")
+                        + " -start " + SchemaHelper.getDateFormat().format(new Date(0))
+                        + " -end " + SchemaHelper.getDateFormat().format(new Date())
+                        + " -filterBy TYPE:PROCESS -orderBy name "
+                        + " -offset 0 -numResults 1 -numInstances 7"));
+        // No start or end date
+        Assert.assertEquals(0,
+                executeWithURL("entity -summary -type process -fields status,pipelines"
+                        + " -cluster " + overlay.get("cluster")
+                        + " -filterBy TYPE:PROCESS -orderBy name "
+                        + " -offset 0 -numResults 1 -numInstances 7"));
+
     }
 
     public void testSubCommandPresence() throws Exception {
@@ -457,6 +476,12 @@ public class FalconCLIIT {
                         + overlay.get("outputFeedName")
                         + " -start "+ SchemaHelper.getDateFormat().format(new Date())
                         +" -filterBy STATUS:SUCCEEDED -offset 0 -numResults 1"));
+        // When you get a cluster for which there are no feed entities,
+        Assert.assertEquals(0,
+                executeWithURL("entity -summary -type feed -cluster " + overlay.get("cluster") + " -fields status,tags"
+                        + " -start "+ SchemaHelper.getDateFormat().format(new Date())
+                        + " -offset 0 -numResults 1 -numInstances 3"));
+
     }
 
     public void testInstanceRunningAndSummaryCommands() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
index 0a41fe0..4755c30 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
@@ -803,6 +803,19 @@ public class EntityManagerJerseyIT {
         for (EntityList.EntityElement entityElement : result.getElements()) {
             Assert.assertNotNull(entityElement.status); // status is null
         }
+
+        response = context.service
+                .path("api/entities/summary/process/" + overlay.get("cluster"))
+                .queryParam("fields", "status,pipelines")
+                .queryParam("numInstances", "1")
+                .queryParam("orderBy", "name")
+                .header("Cookie", context.getAuthenticationToken())
+                .type(MediaType.APPLICATION_JSON)
+                .accept(MediaType.APPLICATION_JSON)
+                .get(ClientResponse.class);
+        Assert.assertEquals(response.getStatus(), 200);
+        EntitySummaryResult summaryResult = response.getEntity(EntitySummaryResult.class);
+        Assert.assertNotNull(summaryResult);
     }
 
     public Date getEndTime() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/305feb0b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/TestContext.java b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
index 9752518..40da789 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
@@ -38,6 +38,7 @@ import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.security.CurrentUser;
+import org.apache.falcon.util.DeploymentUtil;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -350,7 +351,7 @@ public class TestContext {
         long time = System.currentTimeMillis();
         clusterName = "cluster" + time;
         overlay.put("cluster", clusterName);
-        overlay.put("colo", "gs");
+        overlay.put("colo", DeploymentUtil.getCurrentColo());
         overlay.put("inputFeedName", "in" + time);
         //only feeds with future dates can be scheduled
         Date endDate = new Date(System.currentTimeMillis() + 15 * 60 * 1000);
@@ -390,7 +391,7 @@ public class TestContext {
 
         Map<String, String> overlay = new HashMap<String, String>();
         overlay.put("cluster", RandomStringUtils.randomAlphabetic(5));
-        overlay.put("colo", "gs");
+        overlay.put("colo", DeploymentUtil.getCurrentColo());
         TestContext.overlayParametersOverTemplate(clusterTemplate, overlay);
         EmbeddedCluster cluster = EmbeddedCluster.newCluster(overlay.get("cluster"), true);
 


[30/41] git commit: FALCON-657 String datePattern moved to BaseTestClass in falcon-regression

Posted by ra...@apache.org.
FALCON-657 String datePattern moved to BaseTestClass in falcon-regression


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

Branch: refs/heads/FALCON-585
Commit: be08a2e709956c32cb13e0ec66aff729a7ee234b
Parents: cb11ff2
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Thu Sep 4 14:48:19 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Fri Sep 5 10:23:35 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  2 +
 .../regression/testHelper/BaseTestClass.java    |  1 +
 .../falcon/regression/AuthorizationTest.java    | 15 ++++---
 .../regression/ELExp_FutureAndLatestTest.java   |  7 ++--
 .../regression/EmbeddedPigScriptTest.java       |  5 +--
 .../regression/FeedInstanceStatusTest.java      |  2 +-
 .../falcon/regression/FeedReplicationTest.java  |  6 +--
 .../falcon/regression/InstanceParamTest.java    |  3 +-
 .../falcon/regression/InstanceSummaryTest.java  |  3 +-
 .../apache/falcon/regression/NewRetryTest.java  |  4 +-
 .../falcon/regression/NoOutputProcessTest.java  |  2 +-
 .../ProcessInstanceColoMixedTest.java           |  3 +-
 .../regression/ProcessInstanceKillsTest.java    |  5 +--
 .../regression/ProcessInstanceRerunTest.java    |  7 ++--
 .../regression/ProcessInstanceResumeTest.java   |  6 +--
 .../regression/ProcessInstanceRunningTest.java  |  6 +--
 .../regression/ProcessInstanceStatusTest.java   | 12 +++---
 .../regression/ProcessInstanceSuspendTest.java  |  6 +--
 .../falcon/regression/ProcessLibPathTest.java   |  7 ++--
 .../lineage/LineageApiProcessInstanceTest.java  |  5 +--
 .../regression/lineage/LineageApiTest.java      |  5 +--
 .../regression/prism/EntityDryRunTest.java      |  6 +--
 .../prism/FeedDelayParallelTimeoutTest.java     |  2 +-
 .../prism/NewPrismProcessUpdateTest.java        |  2 +-
 .../regression/prism/PrismFeedDeleteTest.java   | 18 +++-----
 .../prism/PrismFeedLateReplicationTest.java     |  3 +-
 .../PrismFeedReplicationPartitionExpTest.java   | 43 ++++++++++----------
 .../prism/PrismFeedReplicationUpdateTest.java   | 13 +++---
 .../regression/prism/PrismFeedSnSTest.java      |  4 +-
 .../regression/prism/PrismFeedUpdateTest.java   |  6 +--
 .../prism/RescheduleKilledProcessTest.java      |  3 +-
 .../RescheduleProcessInFinalStatesTest.java     |  5 +--
 .../prism/UpdateAtSpecificTimeTest.java         |  9 ++--
 .../falcon/regression/ui/LineageGraphTest.java  |  3 +-
 .../falcon/regression/ui/ProcessUITest.java     |  7 ++--
 35 files changed, 99 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 20d5882..aecd520 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,6 +9,8 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
+   FALCON-657 String datePattern moved to BaseTestClass (Ruslan Ostafiychuk)
+
    FALCON-643 Tests with zero-output/input scenario amended to match test case (Paul Isaychuk via
    Ruslan Ostafiychuk)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java b/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
index a951da7..c16c8f4 100644
--- a/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
+++ b/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
@@ -52,6 +52,7 @@ public class BaseTestClass {
     public String baseHDFSDir = "/tmp/falcon-regression";
     public static final String PRISM_PREFIX = "prism";
     protected Bundle[] bundles;
+    public static final String MINUTE_DATE_PATTERN = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
 
 
     public BaseTestClass() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
index fac215f..8bf4288 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
@@ -70,8 +70,7 @@ public class AuthorizationTest extends BaseTestClass {
     private OozieClient clusterOC = serverOC.get(0);
     private String baseTestDir = baseHDFSDir + "/AuthorizationTest";
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    private String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedInputPath = baseTestDir + "/input" + datePattern;
+    private String feedInputPath = baseTestDir + "/input" + MINUTE_DATE_PATTERN;
 
     @BeforeClass(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
@@ -538,8 +537,8 @@ public class AuthorizationTest extends BaseTestClass {
                 .readEntityName(feed)) && !definition.contains("(feed) not found"),
             "Feed should be already submitted");
         //update feed definition
-        String newFeed = Util.setFeedPathValue(feed,
-            baseHDFSDir + "/randomPath/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
+        String newFeed = Util.setFeedPathValue(feed, baseHDFSDir + "/randomPath" +
+            MINUTE_DATE_PATTERN);
         //try to update feed by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
         final ServiceResponse serviceResponse = prism.getFeedHelper().update(feed, newFeed,
@@ -560,8 +559,8 @@ public class AuthorizationTest extends BaseTestClass {
             Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
         //update feed definition
-        String newFeed = Util.setFeedPathValue(feed,
-            baseHDFSDir + "/randomPath/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
+        String newFeed = Util.setFeedPathValue(feed, baseHDFSDir + "/randomPath" +
+            MINUTE_DATE_PATTERN);
         //try to update feed by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
         final ServiceResponse serviceResponse = prism.getFeedHelper().update(feed, newFeed,
@@ -651,8 +650,8 @@ public class AuthorizationTest extends BaseTestClass {
             .getLatestBundleID(cluster, Util.readEntityName(feed), EntityType.FEED);
 
         //update feed definition
-        String newFeed = Util.setFeedPathValue(feed,
-            baseHDFSDir + "/randomPath/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
+        String newFeed = Util.setFeedPathValue(feed, baseHDFSDir + "/randomPath" +
+            MINUTE_DATE_PATTERN);
 
         //update feed by U1
         KerberosHelper.loginFromKeytab(MerlinConstants.CURRENT_USER_NAME);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
index 63484c1..25e2dfe 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
@@ -67,8 +67,7 @@ public class ELExp_FutureAndLatestTest extends BaseTestClass {
         String startDate = TimeUtil.getTimeWrtSystemTime(-20);
         String endDate = TimeUtil.getTimeWrtSystemTime(70);
 
-        b.setInputFeedDataPath(
-            baseTestDir + "/ELExp_latest/testData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        b.setInputFeedDataPath(baseTestDir + "/ELExp_latest/testData" + MINUTE_DATE_PATTERN);
         b.setProcessWorkflow(aggregateWorkflowDir);
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 1);
@@ -83,8 +82,8 @@ public class ELExp_FutureAndLatestTest extends BaseTestClass {
         bundles[0] = BundleUtil.readELBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
-        bundles[0].setInputFeedDataPath(
-            baseTestDir + "/ELExp_latest/testData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setInputFeedDataPath(baseTestDir + "/ELExp_latest/testData" +
+            MINUTE_DATE_PATTERN);
         bundles[0].setInputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setInputFeedValidity("2010-04-01T00:00Z", "2015-04-01T00:00Z");
         String processStart = TimeUtil.getTimeWrtSystemTime(-3);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
index 00f68e6..a0986fa 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
@@ -66,7 +66,7 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
     String pigTestDir = baseHDFSDir + "/EmbeddedPigScriptTest";
     String pigScriptDir = pigTestDir + "/EmbeddedPigScriptTest/pig";
     String pigScriptLocation = pigScriptDir + "/id.pig";
-    String inputPath = pigTestDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String inputPath = pigTestDir + "/input" + MINUTE_DATE_PATTERN;
     private static final Logger logger = Logger.getLogger(EmbeddedPigScriptTest.class);
     private static final double TIMEOUT = 15;
     String processName;
@@ -96,8 +96,7 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
         bundles[0].setInputFeedDataPath(inputPath);
-        bundles[0].setOutputFeedLocationData(
-            pigTestDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setOutputFeedLocationData(pigTestDir + "/output-data" + MINUTE_DATE_PATTERN);
         bundles[0].setProcessWorkflow(pigScriptLocation);
         bundles[0].setProcessInputNames("INPUT");
         bundles[0].setProcessOutputNames("OUTPUT");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
index d4fab30..1e01101 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
@@ -52,7 +52,7 @@ import java.lang.reflect.Method;
 public class FeedInstanceStatusTest extends BaseTestClass {
 
     private String baseTestDir = baseHDFSDir + "/FeedInstanceStatusTest";
-    private String feedInputPath = baseTestDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/";
+    private String feedInputPath = baseTestDir + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
 
     ColoHelper cluster2 = servers.get(1);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
index c4c11cf..fa28a01 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
@@ -23,7 +23,6 @@ import org.apache.falcon.regression.core.bundle.Bundle;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.feed.ActionType;
 import org.apache.falcon.entity.v0.feed.ClusterType;
-import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.regression.core.helpers.ColoHelper;
 import org.apache.falcon.regression.core.response.InstancesResult;
 import org.apache.falcon.regression.core.util.AssertUtil;
@@ -72,12 +71,11 @@ public class FeedReplicationTest extends BaseTestClass {
     private FileSystem cluster3FS = serverFS.get(2);
     private OozieClient cluster2OC = serverOC.get(1);
     private OozieClient cluster3OC = serverOC.get(2);
-    private String dateTemplate = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private String baseTestDir = baseHDFSDir + "/FeedReplicationTest";
     private String sourcePath = baseTestDir + "/source";
-    private String feedDataLocation = baseTestDir + "/source" + dateTemplate;
+    private String feedDataLocation = baseTestDir + "/source" + MINUTE_DATE_PATTERN;
     private String targetPath = baseTestDir + "/target";
-    private String targetDataLocation = targetPath + dateTemplate;
+    private String targetDataLocation = targetPath + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(FeedReplicationTest.class);
 
     @BeforeMethod(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
index edf3428..c56310e 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
@@ -58,8 +58,7 @@ public class InstanceParamTest extends BaseTestClass {
      */
 
     private String baseTestHDFSDir = baseHDFSDir + "/InstanceParamTest";
-    private String feedInputPath = baseTestHDFSDir +
-        "/testInputData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + "/testInputData" + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private String startTime;
     private String endTime;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index 636da2c..75eccaf 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@ -61,8 +61,7 @@ import java.util.List;
 public class InstanceSummaryTest extends BaseTestClass {
 
     String baseTestHDFSDir = baseHDFSDir + "/InstanceSummaryTest";
-    String feedInputPath = baseTestHDFSDir +
-        "/testInputData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String feedInputPath = baseTestHDFSDir + "/testInputData" + MINUTE_DATE_PATTERN;
     String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     String startTime;
     String endTime;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
index 48bf185..c505a93 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
@@ -76,8 +76,8 @@ public class NewRetryTest extends BaseTestClass {
     DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy/MM/dd/HH/mm");
     final private String baseTestDir = baseHDFSDir + "/NewRetryTest";
     final private String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    final private String lateDir = baseTestDir + "/lateDataTest/testFolders/";
-    final private String latePath = lateDir + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    final private String lateDir = baseTestDir + "/lateDataTest/testFolders";
+    final private String latePath = lateDir + MINUTE_DATE_PATTERN;
     private DateTime startDate;
     private DateTime endDate;
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
index 2c30f83..638768d 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
@@ -57,7 +57,7 @@ public class NoOutputProcessTest extends BaseTestClass {
     FileSystem clusterFS = serverFS.get(0);
     OozieClient clusterOC = serverOC.get(0);
     String testDir = baseHDFSDir + "/NoOutputProcessTest";
-    String inputPath = testDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String inputPath = testDir + "/input" + MINUTE_DATE_PATTERN;
     String aggregateWorkflowDir = testDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(NoOutputProcessTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
index 25ef675..9c355a2 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
@@ -56,8 +56,7 @@ import java.util.List;
 public class ProcessInstanceColoMixedTest extends BaseTestClass {
 
     private final String baseTestHDFSDir = baseHDFSDir + "/ProcessInstanceColoMixedTest";
-    private final String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/";
-    private final String feedPath = baseTestHDFSDir + "/feed0%d" + datePattern;
+    private final String feedPath = baseTestHDFSDir + "/feed0%d" + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private ColoHelper cluster1 = servers.get(0);
     private ColoHelper cluster2 = servers.get(1);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
index 33d6b07..ba7a8fd 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
@@ -56,9 +56,8 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     private String testDir = "/ProcessInstanceKillsTest";
     private String baseTestHDFSDir = baseHDFSDir + testDir;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
-    private String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedInputPath = baseTestHDFSDir + "/input" + datePattern;
-    private String feedOutputPath = baseTestHDFSDir + "/output-data" + datePattern;
+    private String feedInputPath = baseTestHDFSDir + "/input" + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceKillsTest.class);
     private static final double TIMEOUT = 15;
     String processName;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
index 3160c1d..8df7f1f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
@@ -54,10 +54,9 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
 
     private String baseTestDir = baseHDFSDir + "/ProcessInstanceRerunTest";
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    private String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedInputPath = baseTestDir + "/input" + datePattern;
-    private String feedOutputPath = baseTestDir + "/output-data" + datePattern;
-    private String feedInputTimedOutPath = baseTestDir + "/timedout" + datePattern;
+    private String feedInputPath = baseTestDir + "/input" + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestDir + "/output-data" + MINUTE_DATE_PATTERN;
+    private String feedInputTimedOutPath = baseTestDir + "/timedout" + MINUTE_DATE_PATTERN;
     private ColoHelper cluster = servers.get(0);
     private FileSystem clusterFS = serverFS.get(0);
     private OozieClient clusterOC = serverOC.get(0);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
index e324c26..fdb64af 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
@@ -53,10 +53,8 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
     private ColoHelper cluster = servers.get(0);
     private FileSystem clusterFS = serverFS.get(0);
     private String baseTestHDFSDir = baseHDFSDir + "/ProcessInstanceResumeTest";
-    private String feedInputPath = baseTestHDFSDir +
-        "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + "/input" + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceResumeTest.class);
     private static final double SCHEDULED = 15;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
index 6bb466c..2b36d4d 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
@@ -57,10 +57,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     private FileSystem clusterFS = serverFS.get(0);
     private String baseTestHDFSDir = baseHDFSDir + "/ProcessInstanceRunningTest";
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
-    private String feedInputPath = baseTestHDFSDir +
-        "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + "/input" + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceRunningTest.class);
     private static final double TIMEOUT = 15;
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
index 5e56037..cac8904 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
@@ -62,14 +62,12 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     private FileSystem clusterFS = serverFS.get(0);
     private String baseTestHDFSDir = baseHDFSDir + "/ProcessInstanceStatusTest";
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
-    private String feedInputPath =
-        baseTestHDFSDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedInputTimedOutPath =
-        baseTestHDFSDir + "/timedoutStatus/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + "/input" + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
+    private String feedInputTimedOutPath = baseTestHDFSDir + "/timedoutStatus" +
+        MINUTE_DATE_PATTERN;
     private String feedOutputTimedOutPath =
-        baseTestHDFSDir + "/output-data/timedoutStatus/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+        baseTestHDFSDir + "/output-data/timedoutStatus" + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceStatusTest.class);
     private static final double TIMEOUT = 15;
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
index 02bd254..80da1ad 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
@@ -56,10 +56,8 @@ import java.util.List;
 public class ProcessInstanceSuspendTest extends BaseTestClass {
 
     private String baseTestHDFSDir = baseHDFSDir + "/ProcessInstanceSuspendTest";
-    private String feedInputPath = baseTestHDFSDir +
-        "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private ColoHelper cluster = servers.get(0);
     private FileSystem clusterFS = serverFS.get(0);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
index fa02fdf..c2d8c9b 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
@@ -54,7 +54,6 @@ public class ProcessLibPathTest extends BaseTestClass {
     String testDir = baseHDFSDir + "/ProcessLibPath";
     String testLibDir = testDir + "/TestLib";
     private static final Logger logger = Logger.getLogger(ProcessLibPathTest.class);
-    String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     String processName;
     String process;
 
@@ -69,7 +68,7 @@ public class ProcessLibPathTest extends BaseTestClass {
         b = new Bundle(b, cluster);
         String startDate = "2010-01-01T22:00Z";
         String endDate = "2010-01-02T03:00Z";
-        b.setInputFeedDataPath(testDir + "/input" + datePattern);
+        b.setInputFeedDataPath(testDir + "/input" + MINUTE_DATE_PATTERN);
         String prefix = b.getFeedDataPathPrefix();
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
@@ -82,11 +81,11 @@ public class ProcessLibPathTest extends BaseTestClass {
         bundles[0] = BundleUtil.readELBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
-        bundles[0].setInputFeedDataPath(baseHDFSDir + datePattern);
+        bundles[0].setInputFeedDataPath(baseHDFSDir + MINUTE_DATE_PATTERN);
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(baseHDFSDir + "/output-data" + datePattern);
+        bundles[0].setOutputFeedLocationData(baseHDFSDir + "/output-data" + MINUTE_DATE_PATTERN);
         bundles[0].setProcessConcurrency(1);
         bundles[0].setProcessLibPath(testLibDir);
         process = bundles[0].getProcessData();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
index 838c6d7..5868882 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
@@ -60,9 +60,8 @@ public class LineageApiProcessInstanceTest extends BaseTestClass {
     String baseTestHDFSDir = baseHDFSDir + "/LineageApiInstanceTest";
     String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     String feedInputPrefix = baseTestHDFSDir + "/input";
-    String feedInputPath = feedInputPrefix + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String feedInputPath = feedInputPrefix + MINUTE_DATE_PATTERN;
+    String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     String processName;
     String inputFeedName;
     String outputFeedName;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
index 37aa98c..0c6ecae 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
@@ -57,7 +57,6 @@ import java.util.TreeMap;
 
 @Test(groups = "lineage-rest")
 public class LineageApiTest extends BaseTestClass {
-    private static final String datePattern = "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private static final Logger logger = Logger.getLogger(LineageApiTest.class);
     private static final String testName = "LineageApiTest";
     private static final String testTag =
@@ -100,7 +99,7 @@ public class LineageApiTest extends BaseTestClass {
         inputMerlin.setTags(testTag);
         inputFeeds = generateFeeds(numInputFeeds, inputMerlin,
             Generator.getNameGenerator("infeed", inputMerlin.getName()),
-            Generator.getHadoopPathGenerator(feedInputPath, datePattern));
+            Generator.getHadoopPathGenerator(feedInputPath, MINUTE_DATE_PATTERN));
         for (FeedMerlin feed : inputFeeds) {
             AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL,
                 feed.toString()));
@@ -110,7 +109,7 @@ public class LineageApiTest extends BaseTestClass {
         outputMerlin.setTags(testTag);
         outputFeeds = generateFeeds(numOutputFeeds, outputMerlin,
             Generator.getNameGenerator("outfeed", outputMerlin.getName()),
-            Generator.getHadoopPathGenerator(feedOutputPath, datePattern));
+            Generator.getHadoopPathGenerator(feedOutputPath, MINUTE_DATE_PATTERN));
         for (FeedMerlin feed : outputFeeds) {
             AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL,
                 feed.toString()));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
index 0b06823..72d03cd 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
@@ -51,10 +51,8 @@ public class EntityDryRunTest extends BaseTestClass {
     private ColoHelper cluster = servers.get(0);
     private FileSystem clusterFS = serverFS.get(0);
     private String baseTestHDFSDir = baseHDFSDir + "/EntityDryRunTest";
-    private String feedInputPath = baseTestHDFSDir +
-        "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String feedOutputPath =
-        baseTestHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String feedInputPath = baseTestHDFSDir + "/input" + MINUTE_DATE_PATTERN;
+    private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
     private static final Logger LOGGER = Logger.getLogger(EntityDryRunTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
index 564637c..c234f0d 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
@@ -46,7 +46,7 @@ public class FeedDelayParallelTimeoutTest extends BaseTestClass {
     ColoHelper cluster2 = servers.get(1);
 
     String baseTestDir = baseHDFSDir + "/FeedDelayParallelTimeoutTest";
-    String feedInputPath = baseTestDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/";
+    String feedInputPath = baseTestDir + MINUTE_DATE_PATTERN;
     String aggregateWorkflowDir = baseTestDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(FeedDelayParallelTimeoutTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
index 884862b..9198901 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
@@ -73,7 +73,7 @@ import java.util.Random;
 public class NewPrismProcessUpdateTest extends BaseTestClass {
 
     private String baseTestDir = baseHDFSDir + "/NewPrismProcessUpdateTest";
-    private String inputFeedPath = baseTestDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private String inputFeedPath = baseTestDir + MINUTE_DATE_PATTERN;
     private String workflowPath = baseTestDir + "/falcon-oozie-wf";
     private String workflowPath2 = baseTestDir + "/falcon-oozie-wf2";
     private String aggregatorPath = baseTestDir + "/aggregator";

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
index b1e78d0..9e3ed12 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
@@ -399,15 +399,13 @@ public class PrismFeedDeleteTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeServer1, "2099-10-01T12:10Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-                "${cluster.colo}",
-                baseHDFSDir + "/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                "${cluster.colo}", baseHDFSDir + "/localDC/rc/billing" + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeServer2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                baseHDFSDir +
-                    "/clusterPath/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                baseHDFSDir + "/clusterPath/localDC/rc/billing" + MINUTE_DATE_PATTERN);
 
         Util.shutDownService(cluster1.getFeedHelper());
 
@@ -836,15 +834,13 @@ public class PrismFeedDeleteTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeServer1, "2099-10-01T12:10Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-                "${cluster.colo}",
-                baseHDFSDir + "/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                "${cluster.colo}", baseHDFSDir + "/localDC/rc/billing" + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeServer2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                baseHDFSDir +
-                    "/clusterPath/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                baseHDFSDir + "/clusterPath/localDC/rc/billing" + MINUTE_DATE_PATTERN);
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
@@ -939,14 +935,12 @@ public class PrismFeedDeleteTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeServer1, "2099-10-01T12:10Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-                "${cluster.colo}",
-                baseHDFSDir + "/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                "${cluster.colo}", baseHDFSDir + "/localDC/rc/billing" + MINUTE_DATE_PATTERN);
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeServer2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                baseHDFSDir + "/clusterPath/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/$" +
-                    "{MINUTE}");
+                baseHDFSDir + "/clusterPath/localDC/rc/billing" + MINUTE_DATE_PATTERN);
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
index 71abd95..8a14229 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
@@ -56,8 +56,7 @@ public class PrismFeedLateReplicationTest extends BaseTestClass {
     FileSystem cluster2FS = serverFS.get(1);
     FileSystem cluster3FS = serverFS.get(2);
     private String baseTestDir = baseHDFSDir + "/PrismFeedLateReplicationTest";
-    private String inputPath =
-        baseTestDir + "/input-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/";
+    private String inputPath = baseTestDir + "/input-data" + MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(PrismFeedLateReplicationTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
index ee7052e..2d67583 100755
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
@@ -72,7 +72,6 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
     private String testDirWithDate = testBaseDir1 + testDate;
     private String testDirWithDate_sourcetarget = testBaseDir4 + testDate;
     private String testDirWithDate_source1 = testBaseDir_server1source + testDate;
-    private String dateTemplate = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private String testFile1 = OSUtil.RESOURCES
         + OSUtil.getPath("ReplicationResources", "feed-s4Replication.xml");
     private String testFile2 = OSUtil.RESOURCES + OSUtil.getPath("ReplicationResources", "id.pig");
@@ -216,13 +215,13 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2012-10-01T12:10Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE, "",
-                testBaseDir1 + dateTemplate);
+                testBaseDir1 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2012-10-01T12:25Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, "",
-                testBaseDir2 + dateTemplate);
+                testBaseDir2 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),
@@ -269,13 +268,13 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(100000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                testBaseDir2 + dateTemplate);
+                testBaseDir2 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),
                 XmlUtil.createRtention("days(100000)", ActionType.DELETE),
                 Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.SOURCE,
-                "${cluster.colo}", testBaseDir1 + dateTemplate);
+                "${cluster.colo}", testBaseDir1 + MINUTE_DATE_PATTERN);
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
@@ -364,13 +363,13 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET,
-                "${cluster.colo}", testBaseDir2 + dateTemplate);
+                "${cluster.colo}", testBaseDir2 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),
             XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
             Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.SOURCE, null,
-            testBaseDir1 + dateTemplate);
+            testBaseDir1 + MINUTE_DATE_PATTERN);
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
@@ -440,7 +439,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
 
         String feed = bundles[0].getDataSets().get(0);
-        feed = InstanceUtil.setFeedFilePath(feed, testBaseDir3 + dateTemplate);
+        feed = InstanceUtil.setFeedFilePath(feed, testBaseDir3 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2010-01-01T00:00Z"),
@@ -547,14 +546,14 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2012-10-01T12:10Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE, null,
-                testBaseDir1 + dateTemplate);
+                testBaseDir1 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2012-10-01T12:25Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET,
                 "${cluster.colo}",
-                testBaseDir2 + dateTemplate);
+                testBaseDir2 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),
@@ -594,7 +593,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         String feed = bundles[0].getDataSets().get(0);
         feed = InstanceUtil.setFeedFilePath(feed,
-            testBaseDir1 + dateTemplate);
+            testBaseDir1 + MINUTE_DATE_PATTERN);
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2010-01-01T00:00Z"),
             XmlUtil.createRtention("days(10000000)", ActionType.DELETE), null,
@@ -604,13 +603,13 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2012-10-01T12:11Z"),
                 XmlUtil.createRtention("days(10000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.TARGET, null,
-                testBaseDir1 + "/ua1" + dateTemplate);
+                testBaseDir1 + "/ua1" + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2012-10-01T12:26Z"),
                 XmlUtil.createRtention("days(10000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                testBaseDir1 + "/ua2" + dateTemplate);
+                testBaseDir1 + "/ua2" + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),
@@ -705,19 +704,19 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
                 "${cluster.colo}",
-                testBaseDir_server1source + dateTemplate);
+                testBaseDir_server1source + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
-                testBaseDir2 + "/replicated" + dateTemplate);
+                testBaseDir2 + "/replicated" + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),
             XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
             Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}", testBaseDir1 + dateTemplate);
+            "${cluster.colo}", testBaseDir1 + MINUTE_DATE_PATTERN);
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
@@ -777,7 +776,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
         String startTimeUA2 = "2012-10-01T12:10Z";
 
         String feed = bundles[0].getDataSets().get(0);
-        feed = InstanceUtil.setFeedFilePath(feed, testBaseDir1 + dateTemplate);
+        feed = InstanceUtil.setFeedFilePath(feed, testBaseDir1 + MINUTE_DATE_PATTERN);
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2010-01-01T00:00Z"),
             XmlUtil.createRtention("days(1000000)", ActionType.DELETE), null,
@@ -787,19 +786,19 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2099-10-01T12:10Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.TARGET,
-                "${cluster.colo}", testBaseDir1 + "/ua1" + dateTemplate + "/");
+                "${cluster.colo}", testBaseDir1 + "/ua1" + MINUTE_DATE_PATTERN + "/");
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET,
-                "${cluster.colo}", testBaseDir1 + "/ua2" + dateTemplate + "/");
+                "${cluster.colo}", testBaseDir1 + "/ua2" + MINUTE_DATE_PATTERN + "/");
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z")
             , XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
             Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.SOURCE,
-            "${cluster.colo}", testBaseDir4 + dateTemplate + "/");
+            "${cluster.colo}", testBaseDir4 + MINUTE_DATE_PATTERN + "/");
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
@@ -876,13 +875,13 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2012-10-01T12:10Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE, "",
-                testBaseDir1 + dateTemplate);
+                testBaseDir1 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2012-10-01T12:25Z"),
                 XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
                 Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.TARGET, "",
-                testBaseDir2 + dateTemplate);
+                testBaseDir2 + MINUTE_DATE_PATTERN);
 
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity("2012-10-01T12:00Z", "2099-01-01T00:00Z"),

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
index 6aebbd7..a4c27d7 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
@@ -59,10 +59,9 @@ public class PrismFeedReplicationUpdateTest extends BaseTestClass {
     String cluster2Colo = cluster2.getClusterHelper().getColoName();
     String cluster3Colo = cluster3.getClusterHelper().getColoName();
     private final String baseTestDir = baseHDFSDir + "/PrismFeedReplicationUpdateTest";
-    private final String inputPath =
-        baseTestDir + "/input-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    private String alternativeInputPath =
-        baseTestDir + "/newFeedPath/input-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    private final String inputPath = baseTestDir + "/input-data" + MINUTE_DATE_PATTERN;
+    private String alternativeInputPath = baseTestDir + "/newFeedPath/input-data" +
+        MINUTE_DATE_PATTERN;
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(PrismFeedReplicationUpdateTest.class);
 
@@ -213,10 +212,8 @@ public class PrismFeedReplicationUpdateTest extends BaseTestClass {
             ClusterType.SOURCE, null);
 
         //set new feed input data
-        feed01 = Util.setFeedPathValue(feed01,
-            baseHDFSDir + "/feed01/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
-        feed02 = Util.setFeedPathValue(feed02,
-            baseHDFSDir + "/feed02/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
+        feed01 = Util.setFeedPathValue(feed01, baseHDFSDir + "/feed01" + MINUTE_DATE_PATTERN);
+        feed02 = Util.setFeedPathValue(feed02, baseHDFSDir + "/feed02" + MINUTE_DATE_PATTERN);
 
         //generate data in both the colos ua1 and ua3
         String prefix = InstanceUtil.getFeedPrefix(feed01);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
index 15a2c3c..71f3751 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
@@ -418,12 +418,12 @@ public class PrismFeedSnSTest extends BaseTestClass {
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA1, "2099-10-01T12:10Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(clust1), ClusterType.SOURCE, "${cluster.colo}",
-                baseHDFSDir + "/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                baseHDFSDir + "/localDC/rc/billing" + MINUTE_DATE_PATTERN);
         feed = InstanceUtil
             .setFeedCluster(feed, XmlUtil.createValidity(startTimeUA2, "2099-10-01T12:25Z"),
                 XmlUtil.createRtention("days(10000)", ActionType.DELETE),
                 Util.readEntityName(clust2), ClusterType.TARGET, null, baseHDFSDir +
-                    "/clusterPath/localDC/rc/billing/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+                    "/clusterPath/localDC/rc/billing" + MINUTE_DATE_PATTERN);
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
         Util.shutDownService(cluster1.getFeedHelper());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
index c0e1617..1225525 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
@@ -66,8 +66,7 @@ public class PrismFeedUpdateTest extends BaseTestClass {
     public final String cluster1colo = cluster1.getClusterHelper().getColoName();
     public final String cluster2colo = cluster2.getClusterHelper().getColoName();
     private static final Logger logger = Logger.getLogger(PrismFeedUpdateTest.class);
-    String feedInputTimedOutPath =
-        baseTestDir + "/timedout/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String feedInputTimedOutPath = baseTestDir + "/timedout" + MINUTE_DATE_PATTERN;
 
     @BeforeClass(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
@@ -131,8 +130,7 @@ public class PrismFeedUpdateTest extends BaseTestClass {
                 ClusterType.SOURCE, null);
 
         //set new feed input data
-        feed01 = Util.setFeedPathValue(feed01,
-            baseTestDir + "/feed01/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
+        feed01 = Util.setFeedPathValue(feed01, baseTestDir + "/feed01" + MINUTE_DATE_PATTERN);
 
         //generate data in both the colos cluster1colo and cluster2colo
         String prefix = InstanceUtil.getFeedPrefix(feed01);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
index 468692a..139b472 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
@@ -113,8 +113,7 @@ public class RescheduleKilledProcessTest extends BaseTestClass {
         bundles[0].setProcessValidity(TimeUtil.getTimeWrtSystemTime(-11),
             TimeUtil.getTimeWrtSystemTime(6));
 
-        bundles[0].setInputFeedDataPath(
-            baseHDFSDir + "/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setInputFeedDataPath(baseHDFSDir + "/rawLogs" + MINUTE_DATE_PATTERN);
 
         logger.info("process: " + Util.prettyPrintXml(bundles[0].getProcessData()));
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
index 7109829..3bba852 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
@@ -56,7 +56,7 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
     FileSystem clusterFS = serverFS.get(0);
     String baseTestDir = baseHDFSDir + "/RescheduleProcessInFinalStates";
     String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    String inputPath = baseTestDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    String inputPath = baseTestDir + "/input" + MINUTE_DATE_PATTERN;
     private static final Logger logger = Logger.getLogger(RescheduleProcessInFinalStatesTest.class);
 
     @BeforeClass(alwaysRun = true)
@@ -92,8 +92,7 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:15Z");
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(
-            baseTestDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setOutputFeedLocationData(baseTestDir + "/output-data" + MINUTE_DATE_PATTERN);
         bundles[0].setProcessConcurrency(6);
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         bundles[0].submitFeedsScheduleProcess(prism);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
index f8a9fd6..cd7d7bd 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
@@ -68,7 +68,6 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
     ColoHelper cluster2 = servers.get(1);
     ColoHelper cluster3 = servers.get(2);
     FileSystem cluster2FS = serverFS.get(1);
-    private String dateTemplate = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private final String baseTestDir = baseHDFSDir + "/UpdateAtSpecificTimeTest-data";
     String aggregateWorkflowDir = baseHDFSDir + "/aggregator";
 
@@ -440,7 +439,7 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         feed = InstanceUtil.setFeedCluster(feed, XmlUtil.createValidity(startTime, endTime),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
             Util.readEntityName(processBundle.getClusters().get(0)), ClusterType.SOURCE,
-            null, baseTestDir + "/replication" + dateTemplate);
+            null, baseTestDir + "/replication" + MINUTE_DATE_PATTERN);
 
         ServiceResponse r = prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
             processBundle.getClusters().get(0));
@@ -585,7 +584,7 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
                 ("2012-10-01T12:10Z", "2099-10-01T12:10Z"),
             XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
             Util.readEntityName(b.getClusters().get(0)), ClusterType.SOURCE, "",
-            "/someTestPath" + dateTemplate);
+            "/someTestPath" + MINUTE_DATE_PATTERN);
         ServiceResponse r = prism.getClusterHelper().submitEntity(Util.URLS
                 .SUBMIT_URL,
             b.getClusters().get(0));
@@ -617,12 +616,12 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             XmlUtil.createValidity(startTimeCluster_target, "2099-10-01T12:25Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
             Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.TARGET,
-            null, testDataDir + dateTemplate);
+            null, testDataDir + MINUTE_DATE_PATTERN);
         feed = InstanceUtil.setFeedCluster(feed,
             XmlUtil.createValidity(startTimeCluster_source, "2099-01-01T00:00Z"),
             XmlUtil.createRtention("days(100000)", ActionType.DELETE),
             Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.SOURCE,
-            null, testDataDir + dateTemplate);
+            null, testDataDir + MINUTE_DATE_PATTERN);
 
         //submit clusters
         Bundle.submitCluster(bundles[0], bundles[1], bundles[2]);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
index 697d8bd..7a69e58 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
@@ -62,8 +62,7 @@ public class LineageGraphTest extends BaseUITestClass {
     private String baseTestDir = baseHDFSDir + "/LineageGraphTest";
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(LineageGraphTest.class);
-    String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    String feedInputPath = baseTestDir + "/input" + datePattern;
+    String feedInputPath = baseTestDir + "/input" + MINUTE_DATE_PATTERN;
     private FileSystem clusterFS = serverFS.get(0);
     private OozieClient clusterOC = serverOC.get(0);
     private String processName = null;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/be08a2e7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
index c82786f..110e2e9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
@@ -69,7 +69,6 @@ public class ProcessUITest extends BaseUITestClass {
     private String baseTestDir = baseHDFSDir + "/TestProcessUI";
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(ProcessUITest.class);
-    String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     String feedInputPath = baseTestDir + "/input";
     final String feedOutputPath = baseTestDir + "/output";
     private FileSystem clusterFS = serverFS.get(0);
@@ -96,7 +95,7 @@ public class ProcessUITest extends BaseUITestClass {
         bundles[0].setProcessPeriodicity(1, Frequency.TimeUnit.minutes);
         bundles[0].setProcessConcurrency(5);
         bundles[0].setInputFeedPeriodicity(1, Frequency.TimeUnit.minutes);
-        bundles[0].setInputFeedDataPath(feedInputPath + datePattern);
+        bundles[0].setInputFeedDataPath(feedInputPath + MINUTE_DATE_PATTERN);
         Process process = bundles[0].getProcessObject();
         Inputs inputs = new Inputs();
         Input input = new Input();
@@ -138,7 +137,7 @@ public class ProcessUITest extends BaseUITestClass {
 
         inputFeeds = LineageApiTest.generateFeeds(numInputFeeds, inputMerlin,
                 Generator.getNameGenerator("infeed", inputMerlin.getName()),
-                Generator.getHadoopPathGenerator(feedInputPath, datePattern));
+                Generator.getHadoopPathGenerator(feedInputPath, MINUTE_DATE_PATTERN));
         int j = 0;
         for (FeedMerlin feed : inputFeeds) {
             bundles[0].addInputFeedToBundle("inputFeed" + j, feed.toString(), j++);
@@ -146,7 +145,7 @@ public class ProcessUITest extends BaseUITestClass {
 
         outputFeeds = LineageApiTest.generateFeeds(numOutputFeeds, outputMerlin,
                 Generator.getNameGenerator("outfeed", outputMerlin.getName()),
-                Generator.getHadoopPathGenerator(feedOutputPath, datePattern));
+                Generator.getHadoopPathGenerator(feedOutputPath, MINUTE_DATE_PATTERN));
         j = 0;
         for (FeedMerlin feed : outputFeeds) {
             bundles[0].addOutputFeedToBundle("outputFeed" + j, feed.toString(), j++);


[39/41] git commit: FALCON-675 Request URLS moved from parameters into methods in falcon-regression

Posted by ra...@apache.org.
FALCON-675 Request URLS moved from parameters into methods in falcon-regression


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

Branch: refs/heads/FALCON-585
Commit: 051a3d29143e0bdaa74469916502114a64b18adb
Parents: 245b6b4
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Fri Sep 5 15:24:56 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Mon Sep 8 16:05:41 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   4 +
 .../falcon/regression/core/bundle/Bundle.java   |  54 +++----
 .../core/helpers/ClusterEntityHelperImpl.java   |  14 +-
 .../core/interfaces/IEntityManagerHelper.java   | 130 +++++++--------
 .../regression/core/supportClasses/Brother.java |  16 +-
 .../falcon/regression/core/util/BundleUtil.java |   3 +-
 .../regression/core/util/CleanupUtil.java       |  11 +-
 .../falcon/regression/core/util/Util.java       |   3 +-
 .../falcon/regression/AuthorizationTest.java    |  78 ++++-----
 .../regression/EmbeddedPigScriptTest.java       |  28 ++--
 .../regression/FeedClusterUpdateTest.java       |  76 ++++-----
 .../regression/FeedInstanceStatusTest.java      |  15 +-
 .../falcon/regression/FeedReplicationTest.java  |  12 +-
 .../falcon/regression/FeedResumeTest.java       |  26 ++-
 .../falcon/regression/FeedScheduleTest.java     |  27 ++--
 .../falcon/regression/FeedStatusTest.java       |  28 ++--
 .../regression/FeedSubmitAndScheduleTest.java   |  26 ++-
 .../falcon/regression/FeedSubmitTest.java       |  21 ++-
 .../falcon/regression/FeedSuspendTest.java      |  28 ++--
 .../falcon/regression/InstanceSummaryTest.java  |   2 +-
 .../apache/falcon/regression/NewRetryTest.java  |  92 ++++++-----
 .../ProcessInstanceColoMixedTest.java           |  11 +-
 .../regression/ProcessInstanceResumeTest.java   |   3 +-
 .../regression/ProcessInstanceRunningTest.java  |  28 ++--
 .../regression/ProcessInstanceStatusTest.java   |  13 +-
 .../regression/ProcessInstanceSuspendTest.java  |   5 +-
 .../regression/hcat/HCatFeedOperationsTest.java |  33 ++--
 .../regression/hcat/HCatReplicationTest.java    |  10 +-
 .../regression/hcat/HCatRetentionTest.java      |   8 +-
 .../regression/lineage/LineageApiTest.java      |  10 +-
 .../regression/prism/EntityDryRunTest.java      |   9 +-
 .../prism/FeedDelayParallelTimeoutTest.java     |   4 +-
 .../prism/NewPrismProcessUpdateTest.java        | 114 +++++--------
 .../prism/PrismClusterDeleteTest.java           |  28 ++--
 .../prism/PrismConcurrentRequestTest.java       |  26 +--
 .../regression/prism/PrismFeedDeleteTest.java   | 109 +++++--------
 .../prism/PrismFeedLateReplicationTest.java     |   9 +-
 .../PrismFeedReplicationPartitionExpTest.java   |  30 ++--
 .../prism/PrismFeedReplicationUpdateTest.java   |  14 +-
 .../regression/prism/PrismFeedResumeTest.java   | 136 ++++++----------
 .../regression/prism/PrismFeedScheduleTest.java |   4 +-
 .../regression/prism/PrismFeedSnSTest.java      | 116 ++++++--------
 .../regression/prism/PrismFeedSuspendTest.java  | 107 ++++---------
 .../regression/prism/PrismFeedUpdateTest.java   |  18 +--
 .../prism/PrismProcessDeleteTest.java           |  94 ++++-------
 .../prism/PrismProcessResumeTest.java           | 159 ++++++-------------
 .../prism/PrismProcessScheduleTest.java         |  53 +++----
 .../regression/prism/PrismProcessSnSTest.java   |  57 +++----
 .../prism/PrismProcessSuspendTest.java          |  97 ++++-------
 .../regression/prism/PrismSubmitTest.java       | 108 +++++--------
 .../prism/RescheduleKilledProcessTest.java      |  31 ++--
 .../RescheduleProcessInFinalStatesTest.java     |  20 ++-
 .../falcon/regression/prism/RetentionTest.java  |   6 +-
 .../prism/UpdateAtSpecificTimeTest.java         |  25 ++-
 .../falcon/regression/ui/LineageGraphTest.java  |   2 +-
 .../falcon/regression/ui/ProcessUITest.java     |   2 +-
 56 files changed, 840 insertions(+), 1353 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 231f33e..b74a4ee 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,8 +9,12 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
+   FALCON-675 Request URLS moved from parameters into methods in falcon-regression (Ruslan
+   Ostafiychuk)
+
    FALCON-656 add test in falcon regression's Authorization test where non-feed owner updates
    a feed with a dependent process(Karishma via Samarth Gupta)
+
    FALCON-674 General code factored out for ProcessInstance* tests (Paul Isaychuk via Ruslan
    Ostafiychuk)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
index 57c7292..6d8b032 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
@@ -46,7 +46,6 @@ import org.apache.falcon.regression.core.response.ServiceResponse;
 import org.apache.falcon.regression.core.util.AssertUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.log4j.Logger;
 import org.joda.time.DateTime;
@@ -80,22 +79,19 @@ public class Bundle {
     public void submitFeed() throws URISyntaxException, IOException, AuthenticationException, JAXBException {
         submitClusters(prismHelper);
 
-        AssertUtil.assertSucceeded(
-            prismHelper.getFeedHelper().submitEntity(URLS.SUBMIT_URL, dataSets.get(0)));
+        AssertUtil.assertSucceeded(prismHelper.getFeedHelper().submitEntity(dataSets.get(0)));
     }
 
     public void submitAndScheduleFeed() throws Exception {
         submitClusters(prismHelper);
 
-        AssertUtil.assertSucceeded(prismHelper.getFeedHelper().submitAndSchedule(
-            URLS.SUBMIT_AND_SCHEDULE_URL, dataSets.get(0)));
+        AssertUtil.assertSucceeded(prismHelper.getFeedHelper().submitAndSchedule(dataSets.get(0)));
     }
 
     public void submitAndScheduleFeedUsingColoHelper(ColoHelper coloHelper) throws Exception {
         submitFeed();
 
-        AssertUtil.assertSucceeded(
-            coloHelper.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, dataSets.get(0)));
+        AssertUtil.assertSucceeded(coloHelper.getFeedHelper().schedule(dataSets.get(0)));
     }
 
     public void submitAndScheduleAllFeeds()
@@ -103,8 +99,7 @@ public class Bundle {
         submitClusters(prismHelper);
 
         for (String feed : dataSets) {
-            AssertUtil.assertSucceeded(
-                prismHelper.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+            AssertUtil.assertSucceeded(prismHelper.getFeedHelper().submitAndSchedule(feed));
         }
     }
 
@@ -112,8 +107,7 @@ public class Bundle {
         IOException, URISyntaxException, AuthenticationException {
         submitClusters(prismHelper);
         submitFeeds(prismHelper);
-        ServiceResponse r = prismHelper.getProcessHelper().submitEntity(URLS.SUBMIT_URL,
-            processData);
+        ServiceResponse r = prismHelper.getProcessHelper().submitEntity(processData);
         if (shouldSucceed) {
             AssertUtil.assertSucceeded(r);
         }
@@ -128,23 +122,20 @@ public class Bundle {
 
         submitFeeds(prismHelper);
 
-        AssertUtil.assertSucceeded(prismHelper.getProcessHelper().submitAndSchedule(
-                URLS.SUBMIT_AND_SCHEDULE_URL, processData));
+        AssertUtil.assertSucceeded(prismHelper.getProcessHelper().submitAndSchedule(processData));
     }
 
 
     public void submitAndScheduleProcess() throws Exception {
         submitAndScheduleAllFeeds();
 
-        AssertUtil.assertSucceeded(prismHelper.getProcessHelper().submitAndSchedule(
-            URLS.SUBMIT_AND_SCHEDULE_URL, processData));
+        AssertUtil.assertSucceeded(prismHelper.getProcessHelper().submitAndSchedule(processData));
     }
 
     public void submitAndScheduleProcessUsingColoHelper(ColoHelper coloHelper) throws Exception {
         submitProcess(true);
 
-        AssertUtil.assertSucceeded(
-                coloHelper.getProcessHelper().schedule(URLS.SCHEDULE_URL, processData));
+        AssertUtil.assertSucceeded(coloHelper.getProcessHelper().schedule(processData));
     }
 
     public List<String> getClusters() {
@@ -271,7 +262,7 @@ public class Bundle {
         //lets submit all data first
         submitFeeds(helper);
 
-        return helper.getProcessHelper().submitEntity(URLS.SUBMIT_URL, getProcessData());
+        return helper.getProcessHelper().submitEntity(getProcessData());
     }
 
     /**
@@ -293,8 +284,7 @@ public class Bundle {
         }
 
         //lets schedule the damn thing now :)
-        ServiceResponse scheduleResult =
-                helper.getProcessHelper().schedule(URLS.SCHEDULE_URL, getProcessData());
+        ServiceResponse scheduleResult = helper.getProcessHelper().schedule(getProcessData());
         AssertUtil.assertSucceeded(scheduleResult);
         TimeUtil.sleepSeconds(7);
         return scheduleResult.getMessage();
@@ -696,16 +686,14 @@ public class Bundle {
     public void submitClusters(ColoHelper helper, String user)
         throws JAXBException, IOException, URISyntaxException, AuthenticationException {
         for (String cluster : this.clusters) {
-            AssertUtil.assertSucceeded(
-                    helper.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster, user));
+            AssertUtil.assertSucceeded(helper.getClusterHelper().submitEntity(cluster, user));
         }
     }
 
     public void submitFeeds(ColoHelper helper)
         throws JAXBException, IOException, URISyntaxException, AuthenticationException {
         for (String feed : this.dataSets) {
-            AssertUtil.assertSucceeded(
-                    helper.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed));
+            AssertUtil.assertSucceeded(helper.getFeedHelper().submitEntity(feed));
         }
     }
 
@@ -757,14 +745,14 @@ public class Bundle {
     public void deleteBundle(ColoHelper helper) {
 
         try {
-            helper.getProcessHelper().delete(URLS.DELETE_URL, getProcessData());
+            helper.getProcessHelper().delete(getProcessData());
         } catch (Exception e) {
             e.getStackTrace();
         }
 
         for (String dataset : getDataSets()) {
             try {
-                helper.getFeedHelper().delete(URLS.DELETE_URL, dataset);
+                helper.getFeedHelper().delete(dataset);
             } catch (Exception e) {
                 e.getStackTrace();
             }
@@ -772,7 +760,7 @@ public class Bundle {
 
         for (String cluster : this.getClusters()) {
             try {
-                helper.getClusterHelper().delete(URLS.DELETE_URL, cluster);
+                helper.getClusterHelper().delete(cluster);
             } catch (Exception e) {
                 e.getStackTrace();
             }
@@ -807,8 +795,7 @@ public class Bundle {
 
         for (Bundle bundle : bundles) {
             ServiceResponse r =
-                prismHelper.getClusterHelper()
-                    .submitEntity(URLS.SUBMIT_URL, bundle.getClusters().get(0));
+                prismHelper.getClusterHelper().submitEntity(bundle.getClusters().get(0));
             Assert.assertTrue(r.getMessage().contains("SUCCEEDED"), r.getMessage());
         }
 
@@ -871,21 +858,18 @@ public class Bundle {
         throws IOException, JAXBException, URISyntaxException, AuthenticationException {
 
         for (int i = 0; i < getClusters().size(); i++) {
-            ServiceResponse r = helper.getClusterHelper()
-                .submitEntity(URLS.SUBMIT_URL, getClusters().get(i));
+            ServiceResponse r = helper.getClusterHelper().submitEntity(getClusters().get(i));
             if (checkSuccess) {
                 AssertUtil.assertSucceeded(r);
             }
         }
         for (int i = 0; i < getDataSets().size(); i++) {
-            ServiceResponse r = helper.getFeedHelper().submitAndSchedule(
-                    URLS.SUBMIT_AND_SCHEDULE_URL, getDataSets().get(i));
+            ServiceResponse r = helper.getFeedHelper().submitAndSchedule(getDataSets().get(i));
             if (checkSuccess) {
                 AssertUtil.assertSucceeded(r);
             }
         }
-        ServiceResponse r = helper.getProcessHelper().submitAndSchedule(
-                URLS.SUBMIT_AND_SCHEDULE_URL, getProcessData());
+        ServiceResponse r = helper.getProcessHelper().submitAndSchedule(getProcessData());
         if (checkSuccess) {
             AssertUtil.assertSucceeded(r);
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/helpers/ClusterEntityHelperImpl.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/helpers/ClusterEntityHelperImpl.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/helpers/ClusterEntityHelperImpl.java
index babf178..49e4b06 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/helpers/ClusterEntityHelperImpl.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/helpers/ClusterEntityHelperImpl.java
@@ -23,7 +23,6 @@ import org.apache.falcon.regression.core.response.InstancesSummaryResult;
 import org.apache.falcon.regression.core.response.InstancesResult;
 import org.apache.falcon.regression.core.response.ServiceResponse;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
@@ -46,29 +45,28 @@ public class ClusterEntityHelperImpl extends IEntityManagerHelper {
         return Util.readEntityName(entity);
     }
 
-    public ServiceResponse getStatus(URLS url, String data, String user) {
+    public ServiceResponse getStatus(String data, String user) {
         throw new UnsupportedOperationException(INVALID_ERR);
     }
 
-    public ServiceResponse resume(URLS url, String data, String user) {
+    public ServiceResponse resume(String data, String user) {
         throw new UnsupportedOperationException(INVALID_ERR);
     }
 
-    public ServiceResponse schedule(URLS url, String data, String user) {
+    public ServiceResponse schedule(String data, String user) {
         throw new UnsupportedOperationException(INVALID_ERR);
     }
 
-    public ServiceResponse submitAndSchedule(URLS url, String data, String user) {
+    public ServiceResponse submitAndSchedule(String data, String user) {
         throw new UnsupportedOperationException(INVALID_ERR);
     }
 
-    public ServiceResponse suspend(URLS url, String data, String user) {
+    public ServiceResponse suspend(String data, String user) {
         throw new UnsupportedOperationException(INVALID_ERR);
     }
 
     @Override
-    public InstancesResult getRunningInstance(
-        URLS processRunningInstance, String name, String user) {
+    public InstancesResult getRunningInstance(String name, String user) {
         throw new UnsupportedOperationException(INVALID_ERR);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/interfaces/IEntityManagerHelper.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/interfaces/IEntityManagerHelper.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/interfaces/IEntityManagerHelper.java
index bf99740..7f8ac6a 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/interfaces/IEntityManagerHelper.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/interfaces/IEntityManagerHelper.java
@@ -253,130 +253,122 @@ public abstract class IEntityManagerHelper {
         return StringUtils.join(parts, "/");
     }
 
-    public ServiceResponse listEntities(URLS url)
+    public ServiceResponse listEntities()
         throws IOException, URISyntaxException, AuthenticationException {
-        return listEntities(url, null);
+        return listEntities(null);
     }
 
-    public ServiceResponse listEntities(Util.URLS url, String user)
+    public ServiceResponse listEntities(String user)
         throws IOException, URISyntaxException, AuthenticationException {
         LOGGER.info("fetching " + getEntityType() + " list");
-        return Util.sendRequest(createUrl(this.hostname + url.getValue(), getEntityType() + colo),
-            "get", null, user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.LIST_URL.getValue(),
+            getEntityType() + colo), "get", null, user);
     }
 
-    public ServiceResponse submitEntity(URLS url, String data)
+    public ServiceResponse submitEntity(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return submitEntity(url, data, null);
+        return submitEntity(data, null);
     }
 
-    public ServiceResponse submitEntity(URLS url, String data, String user)
+    public ServiceResponse submitEntity(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
         LOGGER.info("Submitting " + getEntityType() + ": \n" + Util.prettyPrintXml(data));
-        return Util.sendRequest(createUrl(this.hostname + url.getValue(), getEntityType() + colo),
-            "post", data, user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.SUBMIT_URL.getValue(),
+            getEntityType() + colo), "post", data, user);
     }
 
-    public ServiceResponse schedule(URLS scheduleUrl, String processData)
+    public ServiceResponse schedule(String processData)
         throws IOException, URISyntaxException, AuthenticationException {
-        return schedule(scheduleUrl, processData, null);
+        return schedule(processData, null);
     }
 
-    public ServiceResponse schedule(URLS scheduleUrl, String processData, String user)
+    public ServiceResponse schedule(String processData, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        return Util.sendRequest(createUrl(this.hostname + scheduleUrl.getValue(), getEntityType(),
-            getEntityName(processData) + colo), "post", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.SCHEDULE_URL.getValue(),
+            getEntityType(), getEntityName(processData) + colo), "post", user);
     }
 
-    public ServiceResponse submitAndSchedule(URLS url, String data)
+    public ServiceResponse submitAndSchedule(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return submitAndSchedule(url, data, null);
+        return submitAndSchedule(data, null);
     }
 
-    public ServiceResponse submitAndSchedule(URLS url, String data, String user)
+    public ServiceResponse submitAndSchedule(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
         LOGGER.info("Submitting " + getEntityType() + ": \n" + Util.prettyPrintXml(data));
-        return Util.sendRequest(createUrl(this.hostname + url.getValue(), getEntityType()), "post",
-            data, user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.SUBMIT_AND_SCHEDULE_URL.getValue(),
+            getEntityType()), "post", data, user);
     }
 
-    public ServiceResponse deleteByName(URLS deleteUrl, String entityName, String user)
+    public ServiceResponse deleteByName(String entityName, String user)
         throws AuthenticationException, IOException, URISyntaxException {
-        return Util.sendRequest(
-            createUrl(this.hostname + deleteUrl.getValue(), getEntityType(), entityName + colo),
-            "delete", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.DELETE_URL.getValue(),
+            getEntityType(), entityName + colo), "delete", user);
     }
 
-    public ServiceResponse delete(URLS deleteUrl, String data)
+    public ServiceResponse delete(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return delete(deleteUrl, data, null);
+        return delete(data, null);
     }
 
-    public ServiceResponse delete(URLS deleteUrl, String data, String user)
+    public ServiceResponse delete(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        return Util.sendRequest(
-            createUrl(this.hostname + deleteUrl.getValue(), getEntityType(),
-                getEntityName(data) + colo),
-            "delete", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.DELETE_URL.getValue(),
+            getEntityType(), getEntityName(data) + colo), "delete", user);
     }
 
-    public ServiceResponse suspend(URLS suspendUrl, String data)
+    public ServiceResponse suspend(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return suspend(suspendUrl, data, null);
+        return suspend(data, null);
     }
 
-    public ServiceResponse suspend(URLS url, String data, String user)
+    public ServiceResponse suspend(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        return Util.sendRequest(
-            createUrl(this.hostname + url.getValue(), getEntityType(), getEntityName(data) + colo),
-            "post", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.SUSPEND_URL.getValue(),
+            getEntityType(), getEntityName(data) + colo), "post", user);
     }
 
-    public ServiceResponse resume(URLS url, String data)
+    public ServiceResponse resume(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return resume(url, data, null);
+        return resume(data, null);
     }
 
-    public ServiceResponse resume(URLS url, String data, String user)
+    public ServiceResponse resume(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        return Util.sendRequest(
-            createUrl(this.hostname + url.getValue(), getEntityType(), getEntityName(data) + colo),
-            "post", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.RESUME_URL.getValue(),
+            getEntityType(), getEntityName(data) + colo), "post", user);
     }
 
-    public ServiceResponse getStatus(URLS url, String data)
+    public ServiceResponse getStatus(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return getStatus(url, data, null);
+        return getStatus(data, null);
     }
 
-    public ServiceResponse getStatus(Util.URLS url, String data, String user)
+    public ServiceResponse getStatus(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        return Util.sendRequest(
-            createUrl(this.hostname + url.getValue(), getEntityType(), getEntityName(data) + colo),
-            "get", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.STATUS_URL.getValue(),
+            getEntityType(), getEntityName(data) + colo), "get", user);
     }
 
-    public ServiceResponse getEntityDefinition(URLS url, String data)
+    public ServiceResponse getEntityDefinition(String data)
         throws IOException, URISyntaxException, AuthenticationException {
-        return getEntityDefinition(url, data, null);
+        return getEntityDefinition(data, null);
     }
 
-    public ServiceResponse getEntityDefinition(URLS url, String data, String user)
+    public ServiceResponse getEntityDefinition(String data, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        return Util.sendRequest(
-            createUrl(this.hostname + url.getValue(), getEntityType(), getEntityName(data) + colo),
-            "get", user);
+        return Util.sendRequest(createUrl(this.hostname + URLS.GET_ENTITY_DEFINITION.getValue(),
+            getEntityType(), getEntityName(data) + colo), "get", user);
     }
 
-    public InstancesResult getRunningInstance(URLS processRunningInstance, String name)
+    public InstancesResult getRunningInstance(String name)
         throws IOException, URISyntaxException, AuthenticationException {
-        return getRunningInstance(processRunningInstance, name, null);
+        return getRunningInstance(name, null);
     }
 
-    public InstancesResult getRunningInstance(
-        URLS processRunningInstance, String name, String user)
+    public InstancesResult getRunningInstance(String name, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        String url = createUrl(this.hostname + processRunningInstance.getValue(), getEntityType(),
+        String url = createUrl(this.hostname + URLS.INSTANCE_RUNNING.getValue(), getEntityType(),
             name + allColo);
         return (InstancesResult) InstanceUtil.sendRequestProcessInstance(url, user);
     }
@@ -389,8 +381,8 @@ public abstract class IEntityManagerHelper {
     public InstancesResult getProcessInstanceStatus(
         String entityName, String params, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        String url = createUrl(this.hostname + Util.URLS.INSTANCE_STATUS.getValue(),
-            getEntityType(), entityName, "");
+        String url = createUrl(this.hostname + URLS.INSTANCE_STATUS.getValue(), getEntityType(),
+            entityName, "");
         return (InstancesResult) InstanceUtil
             .createAndSendRequestProcessInstance(url, params, allColo, user);
     }
@@ -404,8 +396,8 @@ public abstract class IEntityManagerHelper {
     public InstancesResult getProcessInstanceSuspend(
         String entityName, String params, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        String url = createUrl(this.hostname + Util.URLS.INSTANCE_SUSPEND.getValue(),
-            getEntityType(), entityName, "");
+        String url = createUrl(this.hostname + URLS.INSTANCE_SUSPEND.getValue(), getEntityType(),
+            entityName, "");
         return (InstancesResult) InstanceUtil
             .createAndSendRequestProcessInstance(url, params, allColo, user);
     }
@@ -417,7 +409,7 @@ public abstract class IEntityManagerHelper {
 
     public ServiceResponse update(String oldEntity, String newEntity, String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        String url = createUrl(this.hostname + Util.URLS.UPDATE.getValue(), getEntityType(),
+        String url = createUrl(this.hostname + URLS.UPDATE.getValue(), getEntityType(),
             getEntityName(oldEntity));
         return Util.sendRequest(url + colo, "post", newEntity, user);
     }
@@ -468,8 +460,8 @@ public abstract class IEntityManagerHelper {
     public InstancesResult getProcessInstanceResume(String entityName, String params,
                                                            String user)
         throws IOException, URISyntaxException, AuthenticationException {
-        String url = createUrl(this.hostname + Util.URLS.INSTANCE_RESUME.getValue(),
-            getEntityType(), entityName, "");
+        String url = createUrl(this.hostname + URLS.INSTANCE_RESUME.getValue(), getEntityType(),
+            entityName, "");
         return (InstancesResult) InstanceUtil
             .createAndSendRequestProcessInstance(url, params, allColo, user);
     }
@@ -504,8 +496,8 @@ public abstract class IEntityManagerHelper {
 
     public InstancesResult getInstanceParams(String entityName, String params)
         throws AuthenticationException, IOException, URISyntaxException {
-        String url = createUrl(this.hostname + Util.URLS.INSTANCE_PARAMS.getValue(),
-            getEntityType(), entityName, "");
+        String url = createUrl(this.hostname + URLS.INSTANCE_PARAMS.getValue(), getEntityType(),
+            entityName, "");
         return (InstancesResult) InstanceUtil
             .createAndSendRequestProcessInstance(url, params, allColo, null);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/supportClasses/Brother.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/supportClasses/Brother.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/supportClasses/Brother.java
index 3fa17e2..5d2708a 100755
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/supportClasses/Brother.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/supportClasses/Brother.java
@@ -76,28 +76,28 @@ public class Brother extends Thread {
         try {
             switch (url) {
             case SUBMIT_URL:
-                output = entityManagerHelper.submitEntity(url, data);
+                output = entityManagerHelper.submitEntity(data);
                 break;
             case GET_ENTITY_DEFINITION:
-                output = entityManagerHelper.getEntityDefinition(url, data);
+                output = entityManagerHelper.getEntityDefinition(data);
                 break;
             case DELETE_URL:
-                output = entityManagerHelper.delete(url, data);
+                output = entityManagerHelper.delete(data);
                 break;
             case SUSPEND_URL:
-                output = entityManagerHelper.suspend(url, data);
+                output = entityManagerHelper.suspend(data);
                 break;
             case SCHEDULE_URL:
-                output = entityManagerHelper.schedule(url, data);
+                output = entityManagerHelper.schedule(data);
                 break;
             case RESUME_URL:
-                output = entityManagerHelper.resume(url, data);
+                output = entityManagerHelper.resume(data);
                 break;
             case SUBMIT_AND_SCHEDULE_URL:
-                output = entityManagerHelper.submitAndSchedule(url, data);
+                output = entityManagerHelper.submitAndSchedule(data);
                 break;
             case STATUS_URL:
-                output = entityManagerHelper.getStatus(url, data);
+                output = entityManagerHelper.getStatus(data);
                 break;
             default:
                 LOGGER.error("Unexpected url: " + url);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
index a069dc5..922c030 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
@@ -118,8 +118,7 @@ public final class BundleUtil {
     public static void submitAllClusters(ColoHelper prismHelper, Bundle... b)
         throws IOException, URISyntaxException, AuthenticationException {
         for (Bundle aB : b) {
-            ServiceResponse r = prismHelper.getClusterHelper()
-                .submitEntity(Util.URLS.SUBMIT_URL, aB.getClusters().get(0));
+            ServiceResponse r = prismHelper.getClusterHelper().submitEntity(aB.getClusters().get(0));
             Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/CleanupUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/CleanupUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/CleanupUtil.java
index 5415083..bf99291 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/CleanupUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/CleanupUtil.java
@@ -73,8 +73,7 @@ public final class CleanupUtil {
     private static EntitiesResult getEntitiesResultOfOneType(
         IEntityManagerHelper iEntityManagerHelper)
         throws IOException, URISyntaxException, AuthenticationException, JAXBException {
-        final ServiceResponse clusterResponse =
-            iEntityManagerHelper.listEntities(Util.URLS.LIST_URL);
+        final ServiceResponse clusterResponse = iEntityManagerHelper.listEntities();
         JAXBContext jc = JAXBContext.newInstance(EntitiesResult.class);
         Unmarshaller u = jc.createUnmarshaller();
         return (EntitiesResult) u.unmarshal(
@@ -86,7 +85,7 @@ public final class CleanupUtil {
             final List<String> clusters = getAllClusters(prism);
             for (String cluster : clusters) {
                 try {
-                    prism.getClusterHelper().deleteByName(Util.URLS.DELETE_URL, cluster, null);
+                    prism.getClusterHelper().deleteByName(cluster, null);
                 } catch (Exception e) {
                     LOGGER.warn("Caught exception: " + ExceptionUtils.getStackTrace(e));
                 }
@@ -103,7 +102,7 @@ public final class CleanupUtil {
             final List<String> feeds = getAllFeeds(prism);
             for (String feed : feeds) {
                 try {
-                    prism.getFeedHelper().deleteByName(Util.URLS.DELETE_URL, feed, null);
+                    prism.getFeedHelper().deleteByName(feed, null);
                 } catch (Exception e) {
                     LOGGER.warn("Caught exception: " + ExceptionUtils.getStackTrace(e));
                 }
@@ -121,7 +120,7 @@ public final class CleanupUtil {
             final List<String> processes = getAllProcesses(prism);
             for (String process : processes) {
                 try {
-                    entityManagerHelper.deleteByName(Util.URLS.DELETE_URL, process, null);
+                    entityManagerHelper.deleteByName(process, null);
                 } catch (Exception e) {
                     LOGGER.warn("Caught exception: " + ExceptionUtils.getStackTrace(e));
                 }
@@ -141,7 +140,7 @@ public final class CleanupUtil {
 
     public static void deleteQuietly(IEntityManagerHelper helper, String feed) {
         try {
-            helper.delete(Util.URLS.DELETE_URL, feed);
+            helper.delete(feed);
         } catch (Exception e) {
             LOGGER.info("Caught exception: " + ExceptionUtils.getStackTrace(e));
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
index 6485784..8fdf65a 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/Util.java
@@ -608,8 +608,7 @@ public final class Util {
         } else {
             helper = cluster.getClusterHelper();
         }
-        ServiceResponse response = helper.getEntityDefinition(URLS
-            .GET_ENTITY_DEFINITION, entity);
+        ServiceResponse response = helper.getEntityDefinition(entity);
         if (shouldReturn) {
             AssertUtil.assertSucceeded(response);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
index 49b13b5..b309ea7 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
@@ -96,7 +96,7 @@ public class AuthorizationTest extends BaseTestClass {
         bundles[0].submitClusters(prism);
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
         final ServiceResponse serviceResponse = prism.getClusterHelper().delete(
-            Util.URLS.DELETE_URL, bundles[0].getClusters().get(0), MerlinConstants.USER2_NAME);
+            bundles[0].getClusters().get(0), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Entity submitted by first user should not be deletable by second user");
     }
@@ -109,7 +109,7 @@ public class AuthorizationTest extends BaseTestClass {
         bundles[0].submitProcess(true);
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
         final ServiceResponse serviceResponse = prism.getProcessHelper().delete(
-            Util.URLS.DELETE_URL, bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
+            bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Entity submitted by first user should not be deletable by second user");
     }
@@ -122,7 +122,7 @@ public class AuthorizationTest extends BaseTestClass {
         bundles[0].submitFeed();
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
         final ServiceResponse serviceResponse = prism.getFeedHelper().delete(
-            Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0), MerlinConstants.USER2_NAME);
+            bundles[0].getDataSets().get(0), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Entity submitted by first user should not be deletable by second user");
     }
@@ -138,8 +138,8 @@ public class AuthorizationTest extends BaseTestClass {
             Job.Status.RUNNING);
         //try to delete process by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getProcessHelper().delete(Util.URLS
-            .DELETE_URL, bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getProcessHelper()
+            .delete(bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Process scheduled by first user should not be deleted by second user");
     }
@@ -151,13 +151,11 @@ public class AuthorizationTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         //submit, schedule feed by U1
         bundles[0].submitClusters(prism);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(
-            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
         //delete feed by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getFeedHelper().delete(Util.URLS
-            .DELETE_URL, feed, MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getFeedHelper().delete(feed, MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Feed scheduled by first user should not be deleted by second user");
     }
@@ -170,14 +168,13 @@ public class AuthorizationTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.SUSPENDED);
         //try to delete process by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getProcessHelper().delete(Util.URLS
-            .DELETE_URL, bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getProcessHelper()
+            .delete(bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Process suspended by first user should not be deleted by second user");
     }
@@ -189,14 +186,13 @@ public class AuthorizationTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         //submit, schedule, suspend feed by U1
         bundles[0].submitClusters(prism);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(
-            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
         //delete feed by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getFeedHelper().delete(Util.URLS
-            .DELETE_URL, feed, MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getFeedHelper()
+            .delete(feed, MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Feed scheduled by first user should not be deleted by second user");
     }
@@ -211,13 +207,12 @@ public class AuthorizationTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         //submit, schedule by U1
         bundles[0].submitClusters(prism);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(
-            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
         //try to suspend by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getFeedHelper().suspend(Util.URLS
-            .SUSPEND_URL, feed, MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getFeedHelper()
+            .suspend(feed, MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Feed scheduled by first user should not be suspended by second user");
     }
@@ -231,8 +226,8 @@ public class AuthorizationTest extends BaseTestClass {
             Job.Status.RUNNING);
         //try to suspend process by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getProcessHelper().suspend(Util.URLS
-            .SUSPEND_URL, bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getProcessHelper()
+            .suspend(bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Process scheduled by first user should not be suspended by second user");
     }
@@ -247,14 +242,13 @@ public class AuthorizationTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         //submit, schedule and then suspend feed by User1
         bundles[0].submitClusters(prism);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(
-            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
         //try to resume feed by User2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getFeedHelper().resume(Util.URLS
-            .RESUME_URL, feed, MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getFeedHelper()
+            .resume(feed, MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Feed suspended by first user should not be resumed by second user");
     }
@@ -265,14 +259,13 @@ public class AuthorizationTest extends BaseTestClass {
     public void u1SuspendU2ResumeProcess() throws Exception {
         //submit, schedule, suspend process by U1
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.SUSPENDED);
         //try to resume process by U2
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        final ServiceResponse serviceResponse = prism.getProcessHelper().resume(Util.URLS
-            .RESUME_URL, bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
+        final ServiceResponse serviceResponse = prism.getProcessHelper()
+            .resume(bundles[0].getProcessData(), MerlinConstants.USER2_NAME);
         AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
             "Process suspended by first user should not be resumed by second user");
     }
@@ -529,10 +522,8 @@ public class AuthorizationTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         //submit feed
         bundles[0].submitClusters(prism);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feed));
-        String definition = prism.getFeedHelper()
-            .getEntityDefinition(Util.URLS.GET_ENTITY_DEFINITION,
-                feed).getMessage();
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed));
+        String definition = prism.getFeedHelper().getEntityDefinition(feed).getMessage();
         Assert.assertTrue(definition.contains(Util
                 .readEntityName(feed)) && !definition.contains("(feed) not found"),
             "Feed should be already submitted");
@@ -555,8 +546,7 @@ public class AuthorizationTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         //submit and schedule feed
         bundles[0].submitClusters(prism);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(
-            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
         //update feed definition
         String newFeed = Util.setFeedPathValue(feed, baseHDFSDir + "/randomPath" +
@@ -579,8 +569,7 @@ public class AuthorizationTest extends BaseTestClass {
         //submit process
         bundles[0].submitBundle(prism);
         String definition = prism.getProcessHelper()
-            .getEntityDefinition(Util.URLS.GET_ENTITY_DEFINITION,
-                bundles[0].getProcessData()).getMessage();
+            .getEntityDefinition(bundles[0].getProcessData()).getMessage();
         Assert.assertTrue(definition.contains(processName)
                 &&
             !definition.contains("(process) not found"), "Process should be already submitted");
@@ -627,14 +616,13 @@ public class AuthorizationTest extends BaseTestClass {
         bundles[0].submitClusters(prism);
         bundles[0].submitFeeds(prism);
         //schedule input feed by U1
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(
-            Util.URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
 
         //by U2 schedule process dependant on scheduled feed by U1
         KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
-        ServiceResponse serviceResponse = prism.getProcessHelper().submitAndSchedule(Util
-            .URLS.SUBMIT_AND_SCHEDULE_URL, process, MerlinConstants.USER2_NAME);
+        ServiceResponse serviceResponse = prism.getProcessHelper()
+            .submitAndSchedule(process, MerlinConstants.USER2_NAME);
         AssertUtil.assertSucceeded(serviceResponse);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
index a0986fa..99ffe37 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
@@ -37,7 +37,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
@@ -126,41 +125,37 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void getResumedProcessInstance() throws Exception {
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
-        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process);
+        prism.getProcessHelper().suspend(process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        ServiceResponse status = prism.getProcessHelper().getStatus(URLS.STATUS_URL, process);
+        ServiceResponse status = prism.getProcessHelper().getStatus(process);
         Assert.assertTrue(status.getMessage().contains("SUSPENDED"), "Process not suspended.");
-        prism.getProcessHelper().resume(URLS.RESUME_URL, process);
+        prism.getProcessHelper().resume(process);
         TimeUtil.sleepSeconds(TIMEOUT);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
     @Test(groups = {"singleCluster"})
     public void getSuspendedProcessInstance() throws Exception {
-        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process);
+        prism.getProcessHelper().suspend(process);
         TimeUtil.sleepSeconds(TIMEOUT);
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
     @Test(groups = {"singleCluster"})
     public void getRunningProcessInstance() throws Exception {
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
     @Test(groups = {"singleCluster"})
     public void getKilledProcessInstance() throws Exception {
-        prism.getProcessHelper().delete(URLS.DELETE_URL, process);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        prism.getProcessHelper().delete(process);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
@@ -168,13 +163,12 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void getSucceededProcessInstance() throws Exception {
         AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
         int counter = OSUtil.IS_WINDOWS ? 100 : 50;
         InstanceUtil.waitForBundleToReachState(cluster, Util.getProcessName(bundles[0]
             .getProcessData()), Job.Status.SUCCEEDED, counter);
-        r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
index 17120e9..3499a4b 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
@@ -30,7 +30,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -126,13 +125,12 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
-        response = cluster2.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOriginalSubmit);
+        response = cluster2.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -184,8 +182,7 @@ public class FeedClusterUpdateTest extends BaseTestClass {
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedUpdated);
+        prism.getFeedHelper().submitAndSchedule(feedUpdated);
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),
                 "REPLICATION"), 0);
@@ -224,13 +221,12 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
-        response = cluster2.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOriginalSubmit);
+        response = cluster2.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -284,8 +280,7 @@ public class FeedClusterUpdateTest extends BaseTestClass {
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedUpdated);
+        prism.getFeedHelper().submitAndSchedule(feedUpdated);
 
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),
@@ -319,13 +314,12 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
-        response = cluster2.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOriginalSubmit);
+        response = cluster2.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -379,8 +373,7 @@ public class FeedClusterUpdateTest extends BaseTestClass {
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedUpdated);
+        prism.getFeedHelper().submitAndSchedule(feedUpdated);
 
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),
@@ -414,14 +407,13 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
 
-        response = cluster2.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOriginalSubmit);
+        response = cluster2.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -473,8 +465,7 @@ public class FeedClusterUpdateTest extends BaseTestClass {
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedUpdated);
+        prism.getFeedHelper().submitAndSchedule(feedUpdated);
 
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),
@@ -508,13 +499,12 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
-        response = cluster2.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOriginalSubmit);
+        response = cluster2.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -568,8 +558,7 @@ public class FeedClusterUpdateTest extends BaseTestClass {
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedUpdated);
+        prism.getFeedHelper().submitAndSchedule(feedUpdated);
 
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),
@@ -614,14 +603,13 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
 
-        response = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOriginalSubmit);
+        response = prism.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -667,12 +655,10 @@ public class FeedClusterUpdateTest extends BaseTestClass {
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
-        response =
-            cluster3.getFeedHelper().getEntityDefinition(URLS.GET_ENTITY_DEFINITION, feedUpdated);
+        response = cluster3.getFeedHelper().getEntityDefinition(feedUpdated);
         AssertUtil.assertFailed(response);
 
-        prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedUpdated);
+        prism.getFeedHelper().submitAndSchedule(feedUpdated);
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),
                 "REPLICATION"), 0);
@@ -746,14 +732,12 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
         logger.info("Feed: " + Util.prettyPrintXml(feedOriginalSubmit));
 
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOriginalSubmit);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feedOriginalSubmit);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(response);
 
         //schedule on source
-        response = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL,
-            feedOriginalSubmit);
+        response = prism.getFeedHelper().schedule(feedOriginalSubmit);
         TimeUtil.sleepSeconds(20);
         AssertUtil.assertSucceeded(response);
 
@@ -805,20 +789,14 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
 
         //verify xmls definitions
-        response =
-            cluster1.getFeedHelper().getEntityDefinition(URLS.GET_ENTITY_DEFINITION, feedUpdated);
+        response = cluster1.getFeedHelper().getEntityDefinition(feedUpdated);
         AssertUtil.assertFailed(response);
-        response = cluster2.getFeedHelper().getEntityDefinition(URLS
-            .GET_ENTITY_DEFINITION, feedUpdated);
+        response = cluster2.getFeedHelper().getEntityDefinition(feedUpdated);
         AssertUtil.assertFailed(response);
-        response = cluster3.getFeedHelper().getEntityDefinition(URLS
-            .GET_ENTITY_DEFINITION, feedUpdated);
-        Assert.assertTrue(XmlUtil.isIdentical(feedUpdated,
-            response.getMessage()));
-        response = prism.getFeedHelper().getEntityDefinition(URLS
-            .GET_ENTITY_DEFINITION, feedUpdated);
-        Assert.assertTrue(XmlUtil.isIdentical(feedUpdated,
-            response.getMessage()));
+        response = cluster3.getFeedHelper().getEntityDefinition(feedUpdated);
+        Assert.assertTrue(XmlUtil.isIdentical(feedUpdated, response.getMessage()));
+        response = prism.getFeedHelper().getEntityDefinition(feedUpdated);
+        Assert.assertTrue(XmlUtil.isIdentical(feedUpdated, response.getMessage()));
 
         Assert.assertEquals(InstanceUtil
             .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feedUpdated),

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
index 1e01101..33a81d0 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
@@ -30,7 +30,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -93,15 +92,15 @@ public class FeedInstanceStatusTest extends BaseTestClass {
 
         logger.info("cluster bundle1: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
         AssertUtil.assertSucceeded(prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0)));
+            .submitEntity(bundles[0].getClusters().get(0)));
 
         logger.info("cluster bundle2: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
         AssertUtil.assertSucceeded(prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0)));
+            .submitEntity(bundles[1].getClusters().get(0)));
 
         logger.info("cluster bundle3: " + Util.prettyPrintXml(bundles[2].getClusters().get(0)));
         AssertUtil.assertSucceeded(prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[2].getClusters().get(0)));
+            .submitEntity(bundles[2].getClusters().get(0)));
 
         String feed = bundles[0].getDataSets().get(0);
         String feedName = Util.readEntityName(feed);
@@ -135,11 +134,11 @@ public class FeedInstanceStatusTest extends BaseTestClass {
             "?start=" + TimeUtil.addMinsToTime(startTime, 100)
                 + "&end=" + TimeUtil.addMinsToTime(startTime, 120));
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed));
         prism.getFeedHelper().getProcessInstanceStatus(feedName,
             "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 100));
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
 
         // both replication instances
         prism.getFeedHelper().getProcessInstanceStatus(feedName,
@@ -215,7 +214,7 @@ public class FeedInstanceStatusTest extends BaseTestClass {
             .addMinsToTime(startTime, 20) + "&end=" + TimeUtil.addMinsToTime(startTime, 40));
 
         //get running instances
-        prism.getFeedHelper().getRunningInstance(URLS.INSTANCE_RUNNING, feedName);
+        prism.getFeedHelper().getRunningInstance(feedName);
 
         //rerun succeeded instance
         prism.getFeedHelper().getProcessInstanceRerun(feedName, "?start=" + startTime);
@@ -237,7 +236,7 @@ public class FeedInstanceStatusTest extends BaseTestClass {
             + "&end=" + TimeUtil.addMinsToTime(startTime, 110));
 
         //kill feed
-        prism.getFeedHelper().delete(URLS.DELETE_URL, feed);
+        prism.getFeedHelper().delete(feed);
         InstancesResult responseInstance = prism.getFeedHelper().getProcessInstanceStatus(feedName,
             "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 110));
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
index fa28a01..197c3c6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
@@ -134,9 +134,7 @@ public class FeedReplicationTest extends BaseTestClass {
 
         //submit and schedule feed
         LOGGER.info("Feed : " + Util.prettyPrintXml(feed));
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
 
         //upload necessary data
         DateTime date = new DateTime(startTime, DateTimeZone.UTC);
@@ -213,9 +211,7 @@ public class FeedReplicationTest extends BaseTestClass {
 
         //submit and schedule feed
         LOGGER.info("Feed : " + Util.prettyPrintXml(feed));
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
 
         //upload necessary data
         DateTime date = new DateTime(startTime, DateTimeZone.UTC);
@@ -305,9 +301,7 @@ public class FeedReplicationTest extends BaseTestClass {
 
         //submit and schedule feed
         LOGGER.info("Feed : " + Util.prettyPrintXml(feed));
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
 
         //upload necessary data
         DateTime date = new DateTime(startTime, DateTimeZone.UTC);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
index a3f37ab..bb629b8 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
@@ -26,7 +26,6 @@ import org.apache.falcon.regression.core.interfaces.IEntityManagerHelper;
 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -74,12 +73,11 @@ public class FeedResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void resumeSuspendedFeed() throws Exception {
-        AssertUtil
-            .assertSucceeded(feedHelper.submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
-        AssertUtil.assertSucceeded(feedHelper.suspend(URLS.SUSPEND_URL, feed));
+        AssertUtil.assertSucceeded(feedHelper.submitAndSchedule(feed));
+        AssertUtil.assertSucceeded(feedHelper.suspend(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(feedHelper.resume(URLS.RESUME_URL, feed));
-        ServiceResponse response = feedHelper.getStatus(URLS.STATUS_URL, feed);
+        AssertUtil.assertSucceeded(feedHelper.resume(feed));
+        ServiceResponse response = feedHelper.getStatus(feed);
         String colo = feedHelper.getColo();
         Assert.assertTrue(response.getMessage().contains(colo + "/RUNNING"));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
@@ -93,7 +91,7 @@ public class FeedResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void resumeNonExistentFeed() throws Exception {
-        AssertUtil.assertFailed(feedHelper.resume(URLS.RESUME_URL, feed));
+        AssertUtil.assertFailed(feedHelper.resume(feed));
     }
 
     /**
@@ -103,10 +101,9 @@ public class FeedResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void resumeDeletedFeed() throws Exception {
-        AssertUtil
-            .assertSucceeded(feedHelper.submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
-        AssertUtil.assertSucceeded(feedHelper.delete(URLS.DELETE_URL, feed));
-        AssertUtil.assertFailed(feedHelper.resume(URLS.RESUME_URL, feed));
+        AssertUtil.assertSucceeded(feedHelper.submitAndSchedule(feed));
+        AssertUtil.assertSucceeded(feedHelper.delete(feed));
+        AssertUtil.assertFailed(feedHelper.resume(feed));
     }
 
     /**
@@ -116,11 +113,10 @@ public class FeedResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void resumeScheduledFeed() throws Exception {
-        AssertUtil
-            .assertSucceeded(feedHelper.submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(feedHelper.submitAndSchedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(feedHelper.resume(URLS.RESUME_URL, feed));
-        ServiceResponse response = feedHelper.getStatus(URLS.STATUS_URL, feed);
+        AssertUtil.assertSucceeded(feedHelper.resume(feed));
+        ServiceResponse response = feedHelper.getStatus(feed);
         String colo = feedHelper.getColo();
         Assert.assertTrue(response.getMessage().contains(colo + "/RUNNING"));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
index 5d87e6e..8500d84 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
@@ -24,7 +24,6 @@ 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -71,15 +70,15 @@ public class FeedScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void scheduleAlreadyScheduledFeed() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        response = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
 
         //now try re-scheduling again
-        response = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        response = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
     }
@@ -92,11 +91,11 @@ public class FeedScheduleTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void scheduleValidFeed() throws Exception {
         //submit feed
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
         //now schedule the thing
-        response = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        response = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
     }
@@ -108,14 +107,13 @@ public class FeedScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void scheduleSuspendedFeed() throws Exception {
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
 
         //now suspend
-        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
         //now schedule this!
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
     }
 
@@ -126,14 +124,13 @@ public class FeedScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void scheduleKilledFeed() throws Exception {
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
 
         //now suspend
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.KILLED);
         //now schedule this!
-        AssertUtil.assertFailed(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertFailed(prism.getFeedHelper().schedule(feed));
     }
 
     /**
@@ -143,7 +140,7 @@ public class FeedScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void scheduleNonExistentFeed() throws Exception {
-        AssertUtil.assertFailed(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertFailed(prism.getFeedHelper().schedule(feed));
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
index 8ce88d4..6278513 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
@@ -26,7 +26,6 @@ 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.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -61,7 +60,7 @@ public class FeedStatusTest extends BaseTestClass {
 
         //submit the cluster
         ServiceResponse response =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+            prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         AssertUtil.assertSucceeded(response);
         feed = bundles[0].getInputFeedFromBundle();
     }
@@ -79,12 +78,11 @@ public class FeedStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getStatusForScheduledFeed() throws Exception {
-        ServiceResponse response =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         LOGGER.info("Feed: " + Util.prettyPrintXml(feed));
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().getStatus(URLS.STATUS_URL, feed);
+        response = prism.getFeedHelper().getStatus(feed);
 
         AssertUtil.assertSucceeded(response);
 
@@ -101,15 +99,14 @@ public class FeedStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getStatusForSuspendedFeed() throws Exception {
-        ServiceResponse response =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
 
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        response = prism.getFeedHelper().suspend(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().getStatus(URLS.STATUS_URL, feed);
+        response = prism.getFeedHelper().getStatus(feed);
 
         AssertUtil.assertSucceeded(response);
         String colo = prism.getFeedHelper().getColo();
@@ -125,11 +122,11 @@ public class FeedStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getStatusForSubmittedFeed() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
 
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().getStatus(URLS.STATUS_URL, feed);
+        response = prism.getFeedHelper().getStatus(feed);
 
         AssertUtil.assertSucceeded(response);
         String colo = prism.getFeedHelper().getColo();
@@ -144,14 +141,13 @@ public class FeedStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getStatusForDeletedFeed() throws Exception {
-        ServiceResponse response =
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().delete(URLS.DELETE_URL, feed);
+        response = prism.getFeedHelper().delete(feed);
         AssertUtil.assertSucceeded(response);
 
-        response = prism.getFeedHelper().getStatus(URLS.STATUS_URL, feed);
+        response = prism.getFeedHelper().getStatus(feed);
         AssertUtil.assertFailed(response);
 
         Assert.assertTrue(
@@ -166,7 +162,7 @@ public class FeedStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getStatusForNonExistentFeed() throws Exception {
-        ServiceResponse response = prism.getFeedHelper().getStatus(URLS.STATUS_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().getStatus(feed);
         AssertUtil.assertFailed(response);
         Assert.assertTrue(
             response.getMessage().contains(Util.readEntityName(feed) + " (FEED) not found"));


[03/41] git commit: FALCON-642 OozieProcessWorkflowBuilderTest test failures. Contributed by Shwetha GS

Posted by ra...@apache.org.
FALCON-642 OozieProcessWorkflowBuilderTest test failures. Contributed by Shwetha GS


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

Branch: refs/heads/FALCON-585
Commit: b7f5915f95cc727701aa372c3c15dc218eea3727
Parents: 06f52ec
Author: Shwetha GS <sh...@inmobi.com>
Authored: Wed Aug 27 09:14:25 2014 +0530
Committer: Shwetha GS <sh...@inmobi.com>
Committed: Wed Aug 27 09:14:25 2014 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                       | 2 ++
 .../falcon/oozie/process/OozieProcessWorkflowBuilderTest.java     | 3 ++-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b7f5915f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 36ca573..b67310b 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -59,6 +59,8 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-642 OozieProcessWorkflowBuilderTest test failures. (Shwetha GS)
+
    FALCON-630 late data rerun for process broken in trunk. (Shwetha GS)
 
    FALCON-611 Post process arg status is in 'FAILED' state always

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b7f5915f/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java b/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
index b1d7a8a..23c01a9 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
@@ -561,7 +561,8 @@ public class OozieProcessWorkflowBuilderTest extends AbstractTestBase {
         Assert.assertEquals(props.get("falconInputFeeds"), process.getInputs().getInputs().get(0).getFeed());
         Assert.assertEquals(props.get("falconInPaths"), "${coord:dataIn('input')}");
         Assert.assertEquals(props.get("falconInputFeedStorageTypes"), Storage.TYPE.TABLE.name());
-        Assert.assertEquals(props.get("falconInputs"), process.getInputs().getInputs().get(0).getName());
+        Assert.assertEquals(props.get(WorkflowExecutionArgs.INPUT_NAMES.getName()),
+            process.getInputs().getInputs().get(0).getName());
 
         // verify the post processing params
         Assert.assertEquals(props.get("feedNames"), process.getOutputs().getOutputs().get(0).getFeed());


[07/41] git commit: FALCON-633 RetryTests and Retentions tests should stop using root dir contributed by Raghav Kumar Gautam

Posted by ra...@apache.org.
FALCON-633 RetryTests and Retentions tests should stop using root dir contributed by Raghav Kumar Gautam


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

Branch: refs/heads/FALCON-585
Commit: ffe18b0ce3092fceffccff215297f6b7e2706550
Parents: 5dfe5cd
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Thu Aug 28 11:36:20 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Thu Aug 28 11:36:20 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  4 +
 .../falcon/regression/core/util/BundleUtil.java |  9 +--
 .../apache/falcon/regression/NewRetryTest.java  | 79 +++++++++++---------
 .../falcon/regression/prism/RetentionTest.java  |  2 +-
 4 files changed, 51 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ffe18b0c/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index faf7e02..7a3b84f 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -8,6 +8,10 @@ Trunk (Unreleased)
    FALCON-589 Add test cases for various feed operations on Hcat feeds (Karishma G 
    via Samarth Gupta)
   IMPROVEMENTS
+
+   FALCON-633 RetryTests and Retentions tests should stop using root dir
+   (Raghav Kumar Gautam via Samarth Gupta)
+
    FALCON-632 Refactoring, documentation stuff (Paul Isaychuk via Samarth Gupta)
 
    FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ffe18b0c/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
index 1f73523..d5790c4 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
@@ -48,14 +48,13 @@ public final class BundleUtil {
         return readBundleFromFolder("LateDataBundles");
     }
 
-    public static Bundle readRetryBundle() throws IOException {
-        return readBundleFromFolder("RetryTests");
+    public static Bundle readRetryBundle(String appPath, String testName) throws IOException {
+        return generateBundleFromTemplate("RetryTests", appPath, testName);
     }
 
-    public static Bundle readRetentionBundle() throws IOException {
-        return readBundleFromFolder("RetentionBundles");
+    public static Bundle readRetentionBundle(String appPath, String testName) throws IOException {
+        return generateBundleFromTemplate("RetentionBundles", appPath, testName);
     }
-
     public static Bundle readELBundle() throws IOException {
         return readBundleFromFolder("ELbundle");
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ffe18b0c/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
index 5ab3dfe..8bcc797 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
@@ -74,8 +74,11 @@ public class NewRetryTest extends BaseTestClass {
     DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy/MM/dd/HH/mm");
     final private String baseTestDir = baseHDFSDir + "/NewRetryTest";
     final private String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    final private String lateDir = baseTestDir + "/lateDataTest/testFolders/";
-    final private String latePath = lateDir + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    final private String lateInputDir = baseTestDir + "/lateDataTest/inputFolders/";
+    final private String lateInputPath = lateInputDir + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    final private String lateOutputDir = baseTestDir + "/lateDataTest/outputFolders/";
+    final private String lateOutputPath = lateOutputDir
+        + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private DateTime startDate;
     private DateTime endDate;
 
@@ -86,15 +89,17 @@ public class NewRetryTest extends BaseTestClass {
 
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
-        bundles[0] = new Bundle(BundleUtil.readRetryBundle(), cluster);
+        bundles[0] = new Bundle(
+            BundleUtil.readRetryBundle(baseAppHDFSDir, this.getClass().getSimpleName()), cluster);
         bundles[0].generateUniqueBundle();
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         startDate = new DateTime(DateTimeZone.UTC).plusMinutes(1);
         endDate = new DateTime(DateTimeZone.UTC).plusMinutes(2);
         bundles[0].setProcessValidity(startDate, endDate);
 
+        bundles[0].setOutputFeedLocationData(lateOutputPath);
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
         feed = Util.insertLateFeedValue(feed, new Frequency("minutes(8)"));
         bundles[0].getDataSets().remove(bundles[0].getInputFeedFromBundle());
         bundles[0].getDataSets().add(feed);
@@ -126,8 +131,8 @@ public class NewRetryTest extends BaseTestClass {
         } else {
             AssertUtil.assertSucceeded(response);
             // lets create data now:
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
 
             //schedule process
             AssertUtil.assertSucceeded(
@@ -177,8 +182,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             //now wait till the process is over
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
@@ -235,8 +240,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
 
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
@@ -288,8 +293,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
 
@@ -344,8 +349,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -392,8 +397,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -444,8 +449,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -498,8 +503,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -550,8 +555,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -589,8 +594,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
 
@@ -643,8 +648,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -680,7 +685,7 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInSuspendedAndResumeCaseWithLateData(Retry retry) throws Exception {
 
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
         feed = Util.insertLateFeedValue(feed, new Frequency("minutes(10)"));
         bundles[0].getDataSets().remove(bundles[0].getInputFeedFromBundle());
         bundles[0].getDataSets().add(feed);
@@ -698,8 +703,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -774,7 +779,7 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInLateDataCase(Retry retry) throws Exception {
 
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
 
         feed = Util.insertLateFeedValue(feed, getFrequency(retry));
 
@@ -795,11 +800,11 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             List<String> initialData =
                 Util.getHadoopDataFromDir(clusterFS, bundles[0].getInputFeedFromBundle(),
-                    lateDir);
+                    lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -837,7 +842,7 @@ public class NewRetryTest extends BaseTestClass {
             String insertionFolder =
                 Util.findFolderBetweenGivenTimeStamps(now, now.plusMinutes(5), initialData);
             logger.info("inserting data in folder " + insertionFolder + " at " + DateTime.now());
-            HadoopUtil.injectMoreData(clusterFS, lateDir + insertionFolder,
+            HadoopUtil.injectMoreData(clusterFS, lateInputDir + insertionFolder,
                     OSUtil.OOZIE_EXAMPLE_INPUT_DATA + "lateData");
             //now to validate all failed instances to check if they were retried or not.
             validateRetry(clusterOC, bundleId,
@@ -854,7 +859,7 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInDeleteAfterPartialRetryCase(Retry retry) throws Exception {
 
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
         feed = Util.insertLateFeedValue(feed, new Frequency("minutes(1)"));
         bundles[0].getDataSets().remove(bundles[0].getInputFeedFromBundle());
         bundles[0].getDataSets().add(feed);
@@ -873,8 +878,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
+            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ffe18b0c/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
index 1d900d9..b288b77 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
@@ -72,7 +72,7 @@ public class RetentionTest extends BaseTestClass {
     @BeforeMethod(alwaysRun = true)
     public void testName(Method method) throws Exception {
         logger.info("test name: " + method.getName());
-        Bundle bundle = BundleUtil.readRetentionBundle();
+        Bundle bundle = BundleUtil.readRetentionBundle(baseAppHDFSDir, this.getClass().getSimpleName());
         bundles[0] = new Bundle(bundle, cluster);
         bundles[0].setInputFeedDataPath(testHDFSDir);
         bundles[0].generateUniqueBundle();


[31/41] git commit: FALCON-674 General code factored out for ProcessInstance* tests. Contributed by Paul Isaychuk

Posted by ra...@apache.org.
FALCON-674 General code factored out for ProcessInstance* tests. Contributed by Paul Isaychuk


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

Branch: refs/heads/FALCON-585
Commit: 1a3728b1f2dfc62c4c8be2bdd6c8c4e663c61955
Parents: be08a2e
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Fri Sep 5 15:11:34 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Fri Sep 5 15:12:32 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   3 +
 .../regression/ProcessInstanceKillsTest.java    |  53 ++---
 .../regression/ProcessInstanceRerunTest.java    |  45 ++---
 .../regression/ProcessInstanceResumeTest.java   | 199 ++++++-------------
 .../regression/ProcessInstanceRunningTest.java  |  48 ++---
 .../regression/ProcessInstanceStatusTest.java   | 136 +++++--------
 .../regression/ProcessInstanceSuspendTest.java  | 107 ++++------
 7 files changed, 196 insertions(+), 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index aecd520..cc38357 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,6 +9,9 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
+   FALCON-674 General code factored out for ProcessInstance* tests (Paul Isaychuk via Ruslan
+   Ostafiychuk)
+
    FALCON-657 String datePattern moved to BaseTestClass (Ruslan Ostafiychuk)
 
    FALCON-643 Tests with zero-output/input scenario amended to match test case (Paul Isaychuk via

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
index ba7a8fd..7b938ec 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
@@ -60,7 +60,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceKillsTest.class);
     private static final double TIMEOUT = 15;
-    String processName;
+    private String processName;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
@@ -85,6 +85,9 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].generateUniqueBundle();
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         bundles[0].setInputFeedDataPath(feedInputPath);
+        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
+        bundles[0].setOutputFeedLocationData(feedOutputPath);
+        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         processName = Util.readEntityName(bundles[0].getProcessData());
     }
 
@@ -103,14 +106,11 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceKillSingle() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
+            "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.KILLED);
     }
 
@@ -127,8 +127,6 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].setProcessConcurrency(2);
         bundles[0].setProcessTimeOut(3, TimeUnit.minutes);
         bundles[0].setProcessPeriodicity(1, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(10);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
@@ -149,8 +147,6 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         bundles[0].setProcessValidity("2010-01-02T00:00Z", "2010-01-02T04:00Z");
         bundles[0].setProcessTimeOut(3, TimeUnit.minutes);
         bundles[0].setProcessPeriodicity(1, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
@@ -181,9 +177,6 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
             baseTestHDFSDir + "/input01", dataDates);
         bundles[0].setInputFeedDataPath(feedInputPath.replace("input/", "input01/"));
         bundles[0].setProcessValidity(startTime, endTime);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         String startTimeRequest = TimeUtil.getTimeWrtSystemTime(-17);
@@ -205,9 +198,6 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
          both start and end r in future with respect to current time
           */
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2099-01-02T01:21Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         String startTime = TimeUtil.getTimeWrtSystemTime(1);
@@ -228,18 +218,15 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceKillMultipleInstance() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:21Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         prism.getProcessHelper()
             .getProcessInstanceKill(processName, "?start=2010-01-02T01:05Z&end=2010-01-02T01:15Z");
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
             "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
-        InstanceUtil.validateResponse(result, 5, 2, 0, 0, 3);
+        InstanceUtil.validateResponse(r, 5, 2, 0, 0, 3);
     }
 
     /**
@@ -251,17 +238,14 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceKillLastInstance() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:21Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
         prism.getProcessHelper().getProcessInstanceKill(processName, "?start=2010-01-02T01:20Z");
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
             "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
-        InstanceUtil.validateResponse(result, 5, 4, 0, 0, 1);
+        InstanceUtil.validateResponse(r, 5, 4, 0, 0, 1);
     }
 
     /**
@@ -273,15 +257,11 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceKillSuspended() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(processName, "?start=2010-01-02T01:00Z");
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
+        prism.getProcessHelper().getProcessInstanceSuspend(processName, "?start=2010-01-02T01:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
+            "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.KILLED);
     }
 
@@ -294,15 +274,12 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceKillSucceeded() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
         InstanceUtil.waitTillInstanceReachState(serverOC.get(0), processName, 1,
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
+            "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.SUCCEEDED);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
index 8df7f1f..4ae6d72 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
@@ -63,6 +63,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceRerunTest.class);
     private static final double TIMEOUT = 10;
     private String processName;
+    private String start = "?start=2010-01-02T01:00Z";
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
@@ -86,6 +87,8 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].generateUniqueBundle();
         bundles[0].setInputFeedDataPath(feedInputPath);
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
+        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         processName = bundles[0].getProcessName();
     }
 
@@ -104,8 +107,6 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunSomeKilled02() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
@@ -113,11 +114,11 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         TimeUtil.sleepSeconds(TIMEOUT);
         InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
         InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:16Z");
+            start + "&end=2010-01-02T01:16Z");
         InstanceUtil.validateResponse(r, 4, 0, 0, 0, 4);
         List<String> wfIDs = InstanceUtil.getWorkflows(cluster, processName);
         prism.getProcessHelper().getProcessInstanceRerun(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            start + "&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 6, 5, 1, 0);
     }
 
@@ -129,8 +130,6 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunMultipleKilled() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:11Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         String process = bundles[0].getProcessData();
@@ -138,11 +137,11 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            .getProcessInstanceKill(processName, start + "&end=2010-01-02T01:11Z");
         InstanceUtil.validateResponse(r, 3, 0, 0, 0, 3);
         List<String> wfIDs =  InstanceUtil.getWorkflows(cluster, processName);
         prism.getProcessHelper().
-            getProcessInstanceRerun(processName, "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            getProcessInstanceRerun(processName, start + "&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 3, 3, 0, 0);
     }
 
@@ -155,19 +154,17 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunSomeKilled01() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
         String process = bundles[0].getProcessData();
         InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
         InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            .getProcessInstanceKill(processName, start + "&end=2010-01-02T01:11Z");
         InstanceUtil.validateResponse(r, 3, 0, 0, 0, 3);
         List<String> wfIDs = InstanceUtil.getWorkflows(cluster, processName);
         prism.getProcessHelper().getProcessInstanceRerun(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            start + "&end=2010-01-02T01:11Z");
         TimeUtil.sleepSeconds(TIMEOUT);
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 6, 6, 0, 0);
     }
@@ -180,16 +177,14 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunSingleKilled() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
         String process = bundles[0].getProcessData();
         InstanceUtil.waitTillInstancesAreCreated(cluster, process, 0);
-        prism.getProcessHelper().getProcessInstanceKill(processName, "?start=2010-01-02T01:00Z");
+        prism.getProcessHelper().getProcessInstanceKill(processName, start);
         String wfID = InstanceUtil.getWorkflows(cluster, processName, Status.KILLED).get(0);
-        prism.getProcessHelper().getProcessInstanceRerun(processName, "?start=2010-01-02T01:00Z");
+        prism.getProcessHelper().getProcessInstanceRerun(processName, start);
         Assert.assertTrue(InstanceUtil.isWorkflowRunning(clusterOC, wfID));
     }
 
@@ -202,8 +197,6 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunSingleSucceeded() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
@@ -213,7 +206,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
                Status.SUCCEEDED).get(0);
         InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 0, CoordinatorAction
             .Status.SUCCEEDED, EntityType.PROCESS);
-        prism.getProcessHelper().getProcessInstanceRerun(processName, "?start=2010-01-02T01:00Z");
+        prism.getProcessHelper().getProcessInstanceRerun(processName, start);
         Assert.assertTrue(InstanceUtil.isWorkflowRunning(clusterOC, wfID));
     }
 
@@ -226,15 +219,13 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunSingleSuspended() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:06Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(2);
         bundles[0].submitFeedsScheduleProcess(prism);
         prism.getProcessHelper().getProcessInstanceSuspend(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:06Z");
+            start + "&end=2010-01-02T01:06Z");
         prism.getProcessHelper().getProcessInstanceRerun(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:06Z");
+            start + "&end=2010-01-02T01:06Z");
         Assert.assertEquals(InstanceUtil.getInstanceStatus(cluster, processName, 0, 1),
             CoordinatorAction.Status.SUSPENDED);
     }
@@ -247,8 +238,6 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceRerunMultipleSucceeded() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:11Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
@@ -258,7 +247,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
         List<String> wfIDs = InstanceUtil.getWorkflows(cluster, processName);
         prism.getProcessHelper().getProcessInstanceRerun(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            start + "&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 3, 3, 0, 0);
     }
 
@@ -272,9 +261,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     public void testProcessInstanceRerunTimedOut() throws Exception {
         bundles[0].setInputFeedDataPath(feedInputTimedOutPath);
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:11Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setProcessTimeOut(2, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
@@ -282,7 +269,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 1,
             CoordinatorAction.Status.TIMEDOUT, EntityType.PROCESS);
         prism.getProcessHelper().getProcessInstanceRerun(processName,
-            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+            start + "&end=2010-01-02T01:11Z");
         s = InstanceUtil.getInstanceStatus(cluster, processName, 0, 0);
         Assert.assertEquals(s, CoordinatorAction.Status.WAITING,
             "instance should have been in WAITING state");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
index fdb64af..bfb8d52 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
@@ -59,6 +59,8 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceResumeTest.class);
     private static final double SCHEDULED = 15;
     private static final double AFFECTED = 10;
+    private String processName;
+    private String wholeRange = "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z";
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
@@ -69,7 +71,6 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
         String startDate = "2010-01-01T23:20Z";
         String endDate = "2010-01-02T01:40Z";
         b.setInputFeedDataPath(feedInputPath);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
             b.getFeedDataPathPrefix(), dataDates);
@@ -84,6 +85,11 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
         bundles[0].setInputFeedDataPath(feedInputPath);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
+        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
+        bundles[0].setProcessConcurrency(6);
+        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
+        processName = Util.readEntityName(bundles[0].getProcessData());
     }
 
     @AfterMethod(alwaysRun = true)
@@ -100,25 +106,17 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeOnlyEnd() throws Exception {
-        bundles[0].setProcessConcurrency(6);
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
+        prism.getProcessHelper().getProcessInstanceSuspend(processName,
+            "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
         TimeUtil.sleepSeconds(AFFECTED);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 2, 4, 0, 0);
-
-        result = prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?end=2010-01-02T01:15Z");
-        InstanceUtil.validateSuccessWithStatusCode(result, ResponseKeys.UNPARSEABLE_DATE);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            wholeRange);
+        InstanceUtil.validateResponse(r, 6, 2, 4, 0, 0);
+        r = prism.getProcessHelper().getProcessInstanceResume(processName, "?end=2010-01-02T01:15Z");
+        InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.UNPARSEABLE_DATE);
     }
 
     /**
@@ -129,28 +127,18 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeResumeSome() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
+        prism.getProcessHelper().getProcessInstanceSuspend(processName,
+            "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
         TimeUtil.sleepSeconds(AFFECTED);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 2, 4, 0, 0);
-
-        prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:16Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 5, 1, 0, 0);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            wholeRange);
+        InstanceUtil.validateResponse(r, 6, 2, 4, 0, 0);
+        prism.getProcessHelper().getProcessInstanceResume(processName,
+            "?start=2010-01-02T01:05Z&end=2010-01-02T01:16Z");
+        r = prism.getProcessHelper().getProcessInstanceStatus(processName, wholeRange);
+        InstanceUtil.validateResponse(r, 6, 5, 1, 0, 0);
     }
 
     /**
@@ -161,28 +149,17 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeResumeMany() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
+        String withinRange = "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z";
+        prism.getProcessHelper().getProcessInstanceSuspend(processName, withinRange);
         TimeUtil.sleepSeconds(AFFECTED);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 2, 4, 0, 0);
-
-        prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 6, 0, 0, 0);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            wholeRange);
+        InstanceUtil.validateResponse(r, 6, 2, 4, 0, 0);
+        prism.getProcessHelper().getProcessInstanceResume(processName, withinRange);
+        r = prism.getProcessHelper().getProcessInstanceStatus(processName, wholeRange);
+        InstanceUtil.validateResponse(r, 6, 6, 0, 0, 0);
     }
 
     /**
@@ -193,22 +170,15 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeSingle() throws Exception {
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        String start = "?start=2010-01-02T01:00Z";
+        prism.getProcessHelper().getProcessInstanceSuspend(processName, start);
         TimeUtil.sleepSeconds(AFFECTED);
-        prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        prism.getProcessHelper().getProcessInstanceResume(processName, start);
         TimeUtil.sleepSeconds(AFFECTED);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName, start);
         InstanceUtil.validateSuccessOnlyStart(r, WorkflowStatus.RUNNING);
     }
 
@@ -220,15 +190,9 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeNonExistent() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r =
-            prism.getProcessHelper()
-                .getProcessInstanceResume("invalidName",
-                    "?start=2010-01-02T01:00Z&end=2010-01-02T01:15Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceResume("invalidName",
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:15Z");
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.PROCESS_NOT_FOUND);
     }
 
@@ -240,13 +204,8 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeNoParams() throws Exception {
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r =
-                prism.getProcessHelper().getProcessInstanceResume(
-                    Util.readEntityName(bundles[0].getProcessData()), null);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceResume(processName, null);
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.UNPARSEABLE_DATE);
     }
 
@@ -258,15 +217,10 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeDeleted() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceResume(processName,
+            "?start=2010-01-02T01:05Z");
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.PROCESS_NOT_FOUND);
     }
 
@@ -277,23 +231,14 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeNonSuspended() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z");
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z");
-        InstanceUtil.validateResponse(result, 1, 1, 0, 0, 0);
-        result = prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z");
-        InstanceUtil.validateResponse(result, 1, 1, 0, 0, 0);
+        String start = "?start=2010-01-02T01:05Z";
+        prism.getProcessHelper().getProcessInstanceResume(processName, start);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName, start);
+        InstanceUtil.validateResponse(r, 1, 1, 0, 0, 0);
+        r = prism.getProcessHelper().getProcessInstanceResume(processName, start);
+        InstanceUtil.validateResponse(r, 1, 1, 0, 0, 0);
     }
 
     /**
@@ -305,28 +250,18 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeLastInstance() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:25Z");
+        String last = "?start=2010-01-02T01:25Z";
+        prism.getProcessHelper().getProcessInstanceSuspend(processName, last);
         TimeUtil.sleepSeconds(AFFECTED);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 5, 1, 0, 0);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            wholeRange);
+        InstanceUtil.validateResponse(r, 6, 5, 1, 0, 0);
         TimeUtil.sleepSeconds(10);
-        prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:25Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 6, 0, 0, 0);
+        prism.getProcessHelper().getProcessInstanceResume(processName, last);
+        r = prism.getProcessHelper().getProcessInstanceStatus(processName, wholeRange);
+        InstanceUtil.validateResponse(r, 6, 6, 0, 0, 0);
     }
 
     /**
@@ -337,27 +272,17 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceResumeWithinRange() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:26Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessConcurrency(6);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(SCHEDULED);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
+        String withinRange = "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z";
+        prism.getProcessHelper().getProcessInstanceSuspend(processName, withinRange);
         TimeUtil.sleepSeconds(AFFECTED);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 2, 4, 0, 0);
-        prism.getProcessHelper()
-            .getProcessInstanceResume(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:05Z&end=2010-01-02T01:21Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:26Z");
-        InstanceUtil.validateResponse(result, 6, 6, 0, 0, 0);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            wholeRange);
+        InstanceUtil.validateResponse(r, 6, 2, 4, 0, 0);
+        prism.getProcessHelper().getProcessInstanceResume(processName, withinRange);
+        r = prism.getProcessHelper().getProcessInstanceStatus(processName, wholeRange);
+        InstanceUtil.validateResponse(r, 6, 6, 0, 0, 0);
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
index 2b36d4d..e92f6b8 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
@@ -61,26 +61,23 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     private String feedOutputPath = baseTestHDFSDir + "/output-data" + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceRunningTest.class);
     private static final double TIMEOUT = 15;
+    private String processName;
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
         LOGGER.info("in @BeforeClass");
         HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-
         Bundle bundle = BundleUtil.readELBundle();
         bundle.generateUniqueBundle();
         bundle = new Bundle(bundle, cluster);
-
         String startDate = "2010-01-02T00:40Z";
         String endDate = "2010-01-02T01:11Z";
-
         bundle.setInputFeedDataPath(feedInputPath);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
             bundle.getFeedDataPathPrefix(), dataDates);
     }
 
-
     @BeforeMethod(alwaysRun = true)
     public void setup(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
@@ -93,6 +90,7 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
+        processName = Util.readEntityName(bundles[0].getProcessData());
     }
 
     @AfterMethod(alwaysRun = true)
@@ -111,15 +109,13 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL,
-            bundles[0].getProcessData()));
+        String process = bundles[0].getProcessData();
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process));
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(URLS.RESUME_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(URLS.RESUME_URL, process));
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
+            processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -136,9 +132,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
         AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL,
             bundles[0].getProcessData()));
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
+            processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
@@ -151,9 +146,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void getRunningProcessInstance() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
+            processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -164,9 +158,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void getNonExistenceProcessInstance() throws Exception {
-        InstancesResult r =
-            prism.getProcessHelper()
-                .getRunningInstance(URLS.INSTANCE_RUNNING, "invalidName");
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
+            "invalidName");
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
@@ -180,9 +173,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     public void getKilledProcessInstance() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
         prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
+            processName);
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
@@ -196,12 +188,10 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void getSucceededProcessInstance() throws Exception {
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitForBundleToReachState(cluster, Util.getProcessName(bundles[0]
-            .getProcessData()), Job.Status.SUCCEEDED);
-        InstancesResult result = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
-        InstanceUtil.validateSuccessWOInstances(result);
+        InstanceUtil.waitForBundleToReachState(cluster, processName, Job.Status.SUCCEEDED);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING,
+            processName);
+        InstanceUtil.validateSuccessWOInstances(r);
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
index cac8904..8c82da9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction.Status;
 import org.apache.oozie.client.Job;
+import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
@@ -70,30 +71,28 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
         baseTestHDFSDir + "/output-data/timedoutStatus" + MINUTE_DATE_PATTERN;
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceStatusTest.class);
     private static final double TIMEOUT = 15;
+    private String processName;
+    private OozieClient clusterOC = serverOC.get(0);
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
         LOGGER.info("in @BeforeClass");
-
         HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-
         Bundle bundle = BundleUtil.readELBundle();
         bundle.generateUniqueBundle();
         bundle = new Bundle(bundle, cluster);
-
         String startDate = "2010-01-01T23:40Z";
         String endDate = "2010-01-02T02:40Z";
-
         bundle.setInputFeedDataPath(feedInputPath);
         String prefix = bundle.getFeedDataPathPrefix();
-
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
     }
 
-
+    /**
+     *  Configures general process definition which particular properties can be overwritten
+     */
     @BeforeMethod(alwaysRun = true)
     public void setup(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
@@ -102,6 +101,9 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
         bundles[0].generateUniqueBundle();
         bundles[0].setInputFeedDataPath(feedInputPath);
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
+        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
+        processName = Util.readEntityName(bundles[0].getProcessData());
     }
 
     @AfterMethod(alwaysRun = true)
@@ -123,9 +125,8 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
         bundles[0].setProcessPeriodicity(1, TimeUnit.minutes);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T10:20Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T10:20Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
         InstanceUtil.validateResponse(r, 6, 1, 0, 5, 0);
     }
@@ -143,9 +144,8 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
         bundles[0].setProcessPeriodicity(1, TimeUnit.minutes);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T05:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T05:00Z");
         AssertUtil.assertSucceeded(r);
         Assert.assertEquals(r.getInstances(), null);
     }
@@ -158,13 +158,10 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusEndOutOfRange() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:30Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:30Z");
         InstanceUtil.validateSuccessWithStatusCode(r, 400);
     }
 
@@ -176,10 +173,8 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     public void testProcessInstanceStatusDateEmpty()
         throws JAXBException, AuthenticationException, IOException, URISyntaxException {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T02:30Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()), null);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName, null);
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.UNPARSEABLE_DATE);
     }
 
@@ -191,11 +186,8 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusStartAndEnd() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
@@ -208,12 +200,9 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusStartOutOfRange() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T00:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateSuccessWithStatusCode(r, 400);
     }
@@ -226,13 +215,10 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusKilled() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
         AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL,
             bundles[0].getProcessData()));
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         if ((r.getStatusCode() != ResponseKeys.PROCESS_NOT_FOUND)) {
             Assert.assertTrue(false);
@@ -247,14 +233,11 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusOnlyStartSuspended() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
         AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL,
             bundles[0].getProcessData()));
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccessOnlyStart(r, WorkflowStatus.SUSPENDED);
     }
@@ -267,11 +250,8 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusReverseDateRange() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:20Z&end=2010-01-02T01:07Z");
         InstanceUtil.validateSuccessWithStatusCode(r, 400);
     }
@@ -284,18 +264,14 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusStartEndOutOfRange() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(
-            feedOutputPath);
+        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(2);
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T00:00Z&end=2010-01-02T01:30Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T00:00Z&end=2010-01-02T01:30Z");
         InstanceUtil.validateSuccessWithStatusCode(r, 400);
     }
 
@@ -307,24 +283,19 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusResumed() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(2);
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
-        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, bundles[0].getProcessData());
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.SUSPENDED);
-        prism.getProcessHelper().resume(URLS.RESUME_URL, bundles[0].getProcessData());
+        String process = bundles[0].getProcessData();
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
+        prism.getProcessHelper().resume(URLS.RESUME_URL, process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:22Z");
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:22Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -337,14 +308,11 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusOnlyStart() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccessOnlyStart(r, WorkflowStatus.RUNNING);
     }
 
@@ -357,7 +325,6 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusInvalidName() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T02:30Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
         InstancesResult r = prism.getProcessHelper()
             .getProcessInstanceStatus("invalidProcess", "?start=2010-01-01T01:00Z");
@@ -374,23 +341,18 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusSuspended() throws Exception {
-        bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         for (int i = 0; i < bundles[0].getClusters().size(); i++) {
             LOGGER.info("cluster to be submitted: " + i + "  "
                     + Util.prettyPrintXml(bundles[0].getClusters().get(i)));
         }
+        String process = bundles[0].getProcessData();
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
-                Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(
-                prism.getProcessHelper().suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
-                Job.Status.SUSPENDED);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(URLS.SUSPEND_URL, process));
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.SUSPENDED);
     }
 
@@ -403,10 +365,8 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceStatusWoParams() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T02:30Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()), null);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName, null);
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.UNPARSEABLE_DATE);
     }
 
@@ -420,17 +380,15 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
     public void testProcessInstanceStatusTimedOut() throws Exception {
         bundles[0].setInputFeedDataPath(feedInputTimedOutPath);
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:11Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setProcessTimeOut(2, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedLocationData(feedOutputTimedOutPath);
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstanceReachState(serverOC.get(0), Util.readEntityName(bundles[0]
-            .getProcessData()), 1, Status.TIMEDOUT, EntityType.PROCESS);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
+        InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 1, Status.TIMEDOUT,
+            EntityType.PROCESS);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.validateFailedInstances(r, 3);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/1a3728b1/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
index 80da1ad..2a4f7f3 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.Job;
+import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
@@ -63,6 +64,8 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     private FileSystem clusterFS = serverFS.get(0);
     private static final Logger LOGGER = Logger.getLogger(ProcessInstanceSuspendTest.class);
     private static final double TIMEOUT = 15;
+    private String processName;
+    private OozieClient clusterOC = serverOC.get(0);
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
@@ -88,6 +91,9 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
         bundles[0].generateUniqueBundle();
         bundles[0].setInputFeedDataPath(feedInputPath);
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
+        bundles[0].setOutputFeedLocationData(feedOutputPath);
+        processName = Util.readEntityName(bundles[0].getProcessData());
     }
 
     @AfterMethod(alwaysRun = true)
@@ -104,23 +110,17 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendLargeRange() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:23Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:21Z");
         InstanceUtil.validateResponse(result, 5, 5, 0, 0, 0);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
+        prism.getProcessHelper().getProcessInstanceSuspend(processName,
                 "?start=2010-01-02T00:00Z&end=2010-01-02T01:30Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        result = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T00:00Z&end=2010-01-02T01:30Z");
         InstanceUtil.validateSuccessWithStatusCode(result, 400);
     }
@@ -134,15 +134,11 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendSucceeded() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(1);
         bundles[0].submitFeedsScheduleProcess(prism);
-        InstanceUtil.waitTillInstanceReachState(serverOC.get(0), Util.getProcessName(bundles[0]
+        InstanceUtil.waitTillInstanceReachState(clusterOC, Util.getProcessName(bundles[0]
             .getProcessData()), 1, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceSuspend(processName,
                 "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccessWithStatusCode(r, 0);
     }
@@ -156,23 +152,17 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendAll() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:23Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateResponse(result, 5, 5, 0, 0, 0);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
+        prism.getProcessHelper().getProcessInstanceSuspend(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        result = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateResponse(result, 5, 0, 5, 0, 0);
     }
@@ -186,15 +176,11 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendWoParams() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:22Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(2);
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()), null);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceSuspend(processName, null);
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.UNPARSEABLE_DATE);
     }
 
@@ -207,23 +193,17 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendStartAndEnd() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:23Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:22Z");
         InstanceUtil.validateResponse(result, 5, 3, 0, 2, 0);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
+        prism.getProcessHelper().getProcessInstanceSuspend(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:15Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
+        result = prism.getProcessHelper().getProcessInstanceStatus(processName,
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:22Z");
         InstanceUtil.validateResponse(result, 5, 0, 3, 2, 0);
     }
@@ -236,15 +216,11 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendNonExistent() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:23Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult r =
-            prism.getProcessHelper()
+        InstancesResult r = prism.getProcessHelper()
                 .getProcessInstanceSuspend("invalidName", "?start=2010-01-02T01:20Z");
         if ((r.getStatusCode() != ResponseKeys.PROCESS_NOT_FOUND)) {
             Assert.assertTrue(false);
@@ -260,24 +236,16 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendOnlyStart() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:11Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(3);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
+        prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING, processName);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceSuspend(processName,
                 "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccessOnlyStart(r, WorkflowStatus.SUSPENDED);
-        prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        prism.getProcessHelper().getRunningInstance(URLS.INSTANCE_RUNNING, processName);
     }
 
     /**
@@ -290,24 +258,17 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     @Test(groups = {"singleCluster"})
     public void testProcessInstanceSuspendSuspendLast() throws Exception {
         bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:23Z");
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(5);
         bundles[0].submitFeedsScheduleProcess(prism);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(serverOC.get(0), EntityType.PROCESS, bundles[0].getProcessData(),
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
             Job.Status.RUNNING);
-        InstancesResult result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        InstancesResult result = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateResponse(result, 5, 5, 0, 0, 0);
-        prism.getProcessHelper()
-            .getProcessInstanceSuspend(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:20Z");
-        result = prism.getProcessHelper()
-            .getProcessInstanceStatus(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        prism.getProcessHelper().getProcessInstanceSuspend(processName, "?start=2010-01-02T01:20Z");
+        result = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
         InstanceUtil.validateResponse(result, 5, 4, 1, 0, 0);
     }
 


[36/41] FALCON-675 Request URLS moved from parameters into methods in falcon-regression

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
index 71f3751..e97ac37 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
@@ -31,7 +31,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -120,13 +119,11 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster2OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         //ensure only one bundle is there
         Assert.assertEquals(OozieUtil.getBundles(cluster1OC,
             Util.readEntityName(feed1), EntityType.FEED).size(), 1);
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
         //ensure only one bundle is there
         Assert.assertEquals(OozieUtil.getBundles(cluster2OC,
             Util.readEntityName(feed2), EntityType.FEED).size(), 1);
@@ -146,28 +143,26 @@ public class PrismFeedSnSTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         Assert.assertEquals(OozieUtil.getBundles(cluster1OC,
             Util.readEntityName(feed1), EntityType.FEED).size(), 1);
 
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper().resume(URLS.RESUME_URL, feed1));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().resume(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         Assert.assertEquals(OozieUtil.getBundles(cluster2OC,
             Util.readEntityName(feed2), EntityType.FEED).size(), 1);
-        AssertUtil.assertSucceeded(cluster2.getFeedHelper().resume(URLS.RESUME_URL, feed2));
+        AssertUtil.assertSucceeded(cluster2.getFeedHelper().resume(feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
@@ -181,18 +176,16 @@ public class PrismFeedSnSTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
     }
 
     /**
@@ -200,10 +193,8 @@ public class PrismFeedSnSTest extends BaseTestClass {
      */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testScheduleNonExistentFeedOnBothColos() throws Exception {
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertFailed(prism.getFeedHelper().submitAndSchedule(feed1));
+        AssertUtil.assertFailed(prism.getFeedHelper().submitAndSchedule(feed2));
     }
 
     /**
@@ -213,12 +204,11 @@ public class PrismFeedSnSTest extends BaseTestClass {
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOn1ColoWhileOtherColoIsDown() throws Exception {
         restartRequired = true;
-        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL,
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(
             bundles[1].getClusters().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
 
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -235,8 +225,7 @@ public class PrismFeedSnSTest extends BaseTestClass {
         restartRequired = true;
         bundles[0].submitFeed();
         Util.shutDownService(cluster1.getFeedHelper());
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
+        AssertUtil.assertFailed(prism.getFeedHelper().submitAndSchedule(feed1));
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
 
@@ -247,8 +236,7 @@ public class PrismFeedSnSTest extends BaseTestClass {
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasSuspendedFeed() throws Exception {
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -264,8 +252,7 @@ public class PrismFeedSnSTest extends BaseTestClass {
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasKilledFeed() throws Exception {
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -282,13 +269,11 @@ public class PrismFeedSnSTest extends BaseTestClass {
     public void testFeedSnSOnBothColosUsingColoHelper() throws Exception {
         //schedule both bundles
         bundles[0].submitFeed();
-        APIResult result = Util.parseResponse((cluster1.getFeedHelper()
-            .submitEntity(URLS.SUBMIT_AND_SCHEDULE_URL, feed1)));
+        APIResult result = Util.parseResponse((cluster1.getFeedHelper().submitEntity(feed1)));
         Assert.assertEquals(result.getStatusCode(), 404);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
         bundles[1].submitFeed();
-        result = Util.parseResponse(cluster2.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        result = Util.parseResponse(cluster2.getFeedHelper().submitAndSchedule(feed2));
         Assert.assertEquals(result.getStatusCode(), 404);
         AssertUtil.checkNotStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
@@ -302,24 +287,20 @@ public class PrismFeedSnSTest extends BaseTestClass {
     public void testSnSSuspendedFeedOnBothColosUsingColoHelper() throws Exception {
         //schedule both bundles
         bundles[0].submitFeed();
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         bundles[1].submitFeed();
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
 
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper().suspend(URLS.SUSPEND_URL, feed1));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().suspend(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
         //now check if they have been scheduled correctly or not
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper().resume(URLS.RESUME_URL, feed1));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().resume(feed1));
 
-        AssertUtil.assertSucceeded(cluster2.getFeedHelper().suspend(URLS.SUSPEND_URL, feed2));
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
+        AssertUtil.assertSucceeded(cluster2.getFeedHelper().suspend(feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
     }
@@ -334,21 +315,19 @@ public class PrismFeedSnSTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed2));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1));
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
 
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(URLS.STATUS_URL,
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(
             feed1)).getMessage(), cluster1.getClusterHelper().getColoName() + "/RUNNING");
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(URLS.STATUS_URL,
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(
             feed2)).getMessage(), cluster2.getClusterHelper().getColoName() + "/RUNNING");
     }
 
@@ -357,11 +336,9 @@ public class PrismFeedSnSTest extends BaseTestClass {
      */
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testSNSNonExistentFeedOnBothColosUsingColoHelper() throws Exception {
-        Assert.assertEquals(Util.parseResponse(cluster1.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed1))
+        Assert.assertEquals(Util.parseResponse(cluster1.getFeedHelper().submitAndSchedule(feed1))
             .getStatusCode(), 404);
-        Assert.assertEquals(Util.parseResponse(cluster2.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2))
+        Assert.assertEquals(Util.parseResponse(cluster2.getFeedHelper().submitAndSchedule(feed2))
             .getStatusCode(), 404);
     }
 
@@ -372,12 +349,11 @@ public class PrismFeedSnSTest extends BaseTestClass {
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testFeedSnSOn1ColoWhileOtherColoIsDownUsingColoHelper() throws Exception {
         restartRequired = true;
-        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL,
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(
             bundles[1].getClusters().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed2));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
 
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -397,12 +373,12 @@ public class PrismFeedSnSTest extends BaseTestClass {
 
         bundles[0].setCLusterColo(cluster1.getClusterHelper().getColoName());
         logger.info("cluster bundles[0]: " + Util.prettyPrintXml(clust1));
-        ServiceResponse r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, clust1);
+        ServiceResponse r = prism.getClusterHelper().submitEntity(clust1);
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         bundles[1].setCLusterColo(cluster2.getClusterHelper().getColoName());
         logger.info("cluster bundles[1]: " + Util.prettyPrintXml(clust2));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, clust2);
+        r = prism.getClusterHelper().submitEntity(clust2);
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         String startTimeUA1 = "2012-10-01T12:00Z";
@@ -428,13 +404,13 @@ public class PrismFeedSnSTest extends BaseTestClass {
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        r = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertPartial(r);
-        r = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertPartial(r);
         Util.startService(cluster1.getFeedHelper());
-        prism.getClusterHelper().delete(URLS.DELETE_URL, clust1);
-        prism.getClusterHelper().delete(URLS.DELETE_URL, clust2);
+        prism.getClusterHelper().delete(clust1);
+        prism.getClusterHelper().delete(clust2);
     }
 
     /**
@@ -445,7 +421,7 @@ public class PrismFeedSnSTest extends BaseTestClass {
     public void testFeedSnSOn1ColoWhileAnotherColoHasSuspendedFeedUsingColoHelper()
         throws Exception {
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper().suspend(URLS.SUSPEND_URL, feed1));
+        AssertUtil.assertSucceeded(cluster1.getFeedHelper().suspend(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
 
         bundles[1].submitAndScheduleFeed();
@@ -462,7 +438,7 @@ public class PrismFeedSnSTest extends BaseTestClass {
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testFeedSnSOn1ColoWhileAnotherColoHasKilledFeedUsingColoHelper() throws Exception {
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(URLS.DELETE_URL, feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed1));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
index 843990f..1b4ad35 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
@@ -82,20 +82,16 @@ public class PrismFeedSuspendTest extends BaseTestClass {
         bundles[1].submitAndScheduleFeed();
 
         //delete using prism
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
         //suspend using prism
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[1].getDataSets().get(0)));
         //suspend on the other one
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
     }
@@ -114,9 +110,7 @@ public class PrismFeedSuspendTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //suspend using prism
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0))
-            );
+                prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
             //verify
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
@@ -124,9 +118,7 @@ public class PrismFeedSuspendTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //suspend on the other one
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0))
-            );
+                prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         }
@@ -139,15 +131,11 @@ public class PrismFeedSuspendTest extends BaseTestClass {
      */
     @Test(groups = "embedded")
     public void testSuspendNonExistentFeedOnBothColos() throws Exception {
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
-
-        AssertUtil.assertFailed(cluster1.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(cluster2.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
+
+        AssertUtil.assertFailed(cluster1.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
     }
 
     /**
@@ -160,15 +148,11 @@ public class PrismFeedSuspendTest extends BaseTestClass {
         bundles[0].submitFeed();
         bundles[1].submitFeed();
 
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
 
-        AssertUtil.assertFailed(cluster1.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(cluster2.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster1.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
     }
 
     /**
@@ -188,17 +172,13 @@ public class PrismFeedSuspendTest extends BaseTestClass {
             Util.shutDownService(cluster1.getFeedHelper());
 
             //suspend using prism
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0))
-            );
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
             //verify
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
             //suspend on the other one
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+                prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         } catch (Exception e) {
             e.printStackTrace();
@@ -224,28 +204,19 @@ public class PrismFeedSuspendTest extends BaseTestClass {
 
             //delete using coloHelpers
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0))
-            );
+                prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
             Util.shutDownService(cluster1.getFeedHelper());
 
             //suspend using prism
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
             //verify
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[1].getDataSets().get(0))
-            );
+                prism.getFeedHelper().delete(bundles[1].getDataSets().get(0)));
             //suspend on the other one
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0))
-            );
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
         } catch (Exception e) {
@@ -272,23 +243,17 @@ public class PrismFeedSuspendTest extends BaseTestClass {
 
             //suspend using prism
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0))
-            );
+                prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
             //verify
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
 
             Util.shutDownService(cluster1.getFeedHelper());
 
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0))
-            );
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
             //suspend on the other one
             AssertUtil.assertSucceeded(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+                prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.SUSPENDED);
         } catch (Exception e) {
@@ -309,16 +274,10 @@ public class PrismFeedSuspendTest extends BaseTestClass {
         throws Exception {
         try {
             Util.shutDownService(cluster1.getFeedHelper());
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
             AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0))
-            );
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-            AssertUtil.assertFailed(
-                cluster2.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+                cluster2.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
         } catch (Exception e) {
             e.printStackTrace();
             throw new TestNGException(e.getCause());
@@ -341,17 +300,11 @@ public class PrismFeedSuspendTest extends BaseTestClass {
 
             Util.shutDownService(cluster1.getFeedHelper());
 
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0))
-            );
-            AssertUtil.assertFailed(
-                prism.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+            AssertUtil.assertFailed(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
 
             AssertUtil.assertFailed(
-                cluster2.getFeedHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+                cluster2.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
         } catch (Exception e) {
             e.printStackTrace();
             throw new TestNGException(e.getCause());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
index 1225525..9c71375 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
@@ -107,10 +107,8 @@ public class PrismFeedUpdateTest extends BaseTestClass {
         logger.info("cluster bundles[1]: " + Util.prettyPrintXml(cluster2Def));
 
         //submit 2 clusters
-        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
-            cluster1Def));
-        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
-            cluster2Def));
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(cluster1Def));
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(cluster2Def));
 
         //get 2 unique feeds
         String feed01 = bundles[0].getInputFeedFromBundle();
@@ -162,10 +160,8 @@ public class PrismFeedUpdateTest extends BaseTestClass {
         //submit and schedule feeds
         logger.info("feed01: " + Util.prettyPrintXml(feed01));
         logger.info("outputFeed: " + Util.prettyPrintXml(outputFeed));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, feed01));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, outputFeed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed01));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(outputFeed));
 
         /* create 2 process with 2 clusters */
         //get first process
@@ -197,10 +193,8 @@ public class PrismFeedUpdateTest extends BaseTestClass {
         //submit and schedule both process
         logger.info("process: " + Util.prettyPrintXml(process01));
         logger.info("process: " + Util.prettyPrintXml(process02));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, process01));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, process02));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process01));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process02));
         logger.info("Wait till process goes into running ");
         InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(process01), 1,
             CoordinatorAction.Status.RUNNING, EntityType.PROCESS, 1);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
index 18c29f3..1bc61b3 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
@@ -26,7 +26,6 @@ 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.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
@@ -95,8 +94,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         List<String> initialUA2ArchiveStore = cluster1.getProcessHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -147,8 +145,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             Util.shutDownService(cluster2.getClusterHelper());
 
             //lets now delete the cluster from both colos
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -180,8 +177,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             //bring service up
             Util.startService(cluster2.getProcessHelper());
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             HashMap<String, List<String>> systemPostUp = getSystemState(EntityType.PROCESS);
 
@@ -210,9 +206,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             //now submit the thing to prism
             bundles[0].submitFeedsScheduleProcess();
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
             //fetch the initial store and archive state for prism
             List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
             List<String> initialPrismArchiveStore = prism.getProcessHelper().getArchiveInfo();
@@ -225,9 +219,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             List<String> initialUA2ArchiveStore = cluster1.getProcessHelper().getArchiveInfo();
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -268,8 +260,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             Util.shutDownService(cluster2.getClusterHelper());
 
             //lets now delete the cluster from both colos
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //now lets get the final states
             List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -287,8 +278,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //delete again
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //get final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -341,8 +331,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //delete
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //get final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -396,9 +385,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //delete
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //get final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -428,9 +415,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             Util.startService(cluster2.getClusterHelper());
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         } catch (Exception e) {
             logger.info(e.getMessage());
@@ -458,8 +443,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         List<String> initialUA2ArchiveStore = cluster1.getProcessHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -496,8 +480,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         bundles[1].submitFeedsScheduleProcess();
 
         //suspend UA1 colo thingy
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -511,8 +494,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         List<String> initialUA2ArchiveStore = cluster1.getProcessHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -550,10 +532,8 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         bundles[1].submitFeedsScheduleProcess();
 
         //suspend UA1 colo thingy
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -567,8 +547,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         List<String> initialUA2ArchiveStore = cluster1.getProcessHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -606,9 +585,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             bundles[1].submitFeedsScheduleProcess();
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().suspend(bundles[0].getProcessData()));
 
             //fetch the initial store and archive state for prism
             List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -625,8 +602,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             Util.shutDownService(cluster2.getFeedHelper());
 
             //lets now delete the cluster from both colos
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -684,8 +660,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             Util.shutDownService(cluster2.getFeedHelper());
 
             //lets now delete the cluster from both colos
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -716,9 +691,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             Util.startService(cluster2.getClusterHelper());
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             HashMap<String, List<String>> systemPostUp = getSystemState(EntityType.PROCESS);
 
@@ -750,9 +723,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //now submit the thing to prism
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().suspend(bundles[1].getProcessData()));
             //fetch the initial store and archive state for prism
             List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
             List<String> initialPrismArchiveStore = prism.getProcessHelper().getArchiveInfo();
@@ -769,9 +740,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //lets now delete the cluster from both colos
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[1].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -817,12 +786,9 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //now submit the thing to prism
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().suspend(bundles[1].getProcessData()));
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().suspend(bundles[0].getProcessData()));
             //fetch the initial store and archive state for prism
             List<String> initialPrismStore = prism.getProcessHelper().getStoreInfo();
             List<String> initialPrismArchiveStore = prism.getProcessHelper().getArchiveInfo();
@@ -839,9 +805,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //lets now delete the cluster from both colos
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[1].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -900,9 +864,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
 
             //lets now delete the cluster from both colos
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[1].getProcessData()));
 
             //now lets get the final states
             List<String> finalPrismStore = prism.getProcessHelper().getStoreInfo();
@@ -935,9 +897,7 @@ public class PrismProcessDeleteTest extends BaseTestClass {
             Util.startService(cluster2.getClusterHelper());
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             HashMap<String, List<String>> systemPostUp = getSystemState(EntityType.PROCESS);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
index 4ac57fc..c2b9681 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
@@ -82,46 +82,38 @@ public class PrismProcessResumeTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcessUsingColoHelper(cluster1);
 
         //suspend using prism
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //resume using prism
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //suspend using the colohelper
         AssertUtil.assertSucceeded(
-            cluster2.getProcessHelper()
-                .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData())
-        );
+            cluster2.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //resume using colohelper
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //suspend on the other one
         AssertUtil.assertSucceeded(
-            cluster1.getProcessHelper()
-                .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData())
-        );
+            cluster1.getProcessHelper().suspend(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
 
         //resume using colohelper
-        AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
     }
@@ -139,33 +131,27 @@ public class PrismProcessResumeTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcessUsingColoHelper(cluster1);
 
         //delete using prism
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         //try to resume it through prism
-        AssertUtil.assertFailed(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //delete using prism
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .delete(Util.URLS.DELETE_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[1].getProcessData()));
 
         //try to resume it through prism
-        AssertUtil.assertFailed(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
 
         //try to resume process through colohelper
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         //try to resume process through colohelper
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
     }
 
@@ -181,30 +167,25 @@ public class PrismProcessResumeTest extends BaseTestClass {
         bundles[0].submitAndScheduleProcessUsingColoHelper(cluster2);
         bundles[1].submitAndScheduleProcessUsingColoHelper(cluster1);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
 
         for (int i = 0; i < 2; i++) {
             //resume suspended process using prism
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+                prism.getProcessHelper().resume(bundles[0].getProcessData()));
             //verify
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         }
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
 
         for (int i = 0; i < 2; i++) {
             //resume resumed process
             AssertUtil.assertSucceeded(
-                cluster2.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData())
-            );
+                cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
             //verify
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
             AssertUtil
@@ -214,9 +195,7 @@ public class PrismProcessResumeTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //resume on the other one
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().resume(bundles[1].getProcessData()));
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         }
@@ -224,9 +203,7 @@ public class PrismProcessResumeTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //resume another resumed process
             AssertUtil.assertSucceeded(
-                cluster1.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData())
-            );
+                cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         }
@@ -239,15 +216,11 @@ public class PrismProcessResumeTest extends BaseTestClass {
      */
     @Test
     public void testResumeNonExistentProcessOnBothColos() throws Exception {
-        AssertUtil.assertFailed(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
-
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[1].getProcessData()));
+
+        AssertUtil.assertFailed(cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().resume(bundles[0].getProcessData()));
     }
 
     /**
@@ -260,15 +233,11 @@ public class PrismProcessResumeTest extends BaseTestClass {
         bundles[0].submitProcess(true);
         bundles[1].submitProcess(true);
 
-        AssertUtil.assertFailed(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(prism.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[1].getProcessData()));
 
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
 
 
     }
@@ -287,26 +256,20 @@ public class PrismProcessResumeTest extends BaseTestClass {
             bundles[0].submitAndScheduleProcessUsingColoHelper(cluster2);
             bundles[1].submitAndScheduleProcessUsingColoHelper(cluster1);
             AssertUtil.assertSucceeded(
-                cluster2.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData())
-            );
+                cluster2.getProcessHelper().suspend(bundles[0].getProcessData()));
             AssertUtil.assertSucceeded(
-                cluster1.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData())
-            );
+                cluster1.getProcessHelper().suspend(bundles[1].getProcessData()));
 
             Util.shutDownService(cluster2.getProcessHelper());
 
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
             //verify
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
 
             //resume on the other one
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+                prism.getProcessHelper().resume(bundles[1].getProcessData()));
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
             AssertUtil
                 .checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
@@ -335,42 +298,32 @@ public class PrismProcessResumeTest extends BaseTestClass {
 
             //delete using coloHelpers
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[0].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
             Util.shutDownService(cluster2.getProcessHelper());
 
             //try to resume using prism
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
             //verify
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
             //try to resume using colohelper
             AssertUtil.assertFailed(
-                cluster2.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData())
-            );
+                cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
             //verify
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .delete(Util.URLS.DELETE_URL, bundles[1].getProcessData())
-            );
+                prism.getProcessHelper().delete(bundles[1].getProcessData()));
             //suspend on the other one
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[1].getProcessData()));
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
 
             AssertUtil.assertFailed(
-                cluster1.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData())
-            );
+                cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
         } catch (Exception e) {
@@ -397,34 +350,28 @@ public class PrismProcessResumeTest extends BaseTestClass {
 
             //suspend using prism
             AssertUtil.assertSucceeded(
-                cluster2.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData())
-            );
+                cluster2.getProcessHelper().suspend(bundles[0].getProcessData()));
             //verify
             AssertUtil
                 .checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
             AssertUtil.assertSucceeded(
-                cluster2.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+                cluster2.getProcessHelper().resume(bundles[0].getProcessData()));
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
             Util.shutDownService(cluster2.getProcessHelper());
 
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
 
 
             AssertUtil.assertSucceeded(
-                prism.getProcessHelper()
-                    .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                prism.getProcessHelper().suspend(bundles[1].getProcessData()));
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
 
             for (int i = 0; i < 2; i++) {
                 //suspend on the other one
                 AssertUtil.assertSucceeded(
-                    prism.getProcessHelper()
-                        .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+                    prism.getProcessHelper().resume(bundles[1].getProcessData()));
                 AssertUtil
                     .checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
                 AssertUtil
@@ -434,8 +381,7 @@ public class PrismProcessResumeTest extends BaseTestClass {
             for (int i = 0; i < 2; i++) {
                 //suspend on the other one
                 AssertUtil.assertSucceeded(
-                    cluster1.getProcessHelper()
-                        .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+                    cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
                 AssertUtil
                     .checkStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
                 AssertUtil
@@ -462,13 +408,10 @@ public class PrismProcessResumeTest extends BaseTestClass {
         try {
             Util.shutDownService(cluster2.getProcessHelper());
 
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[1].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
             AssertUtil.assertFailed(
-                cluster1.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+                cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
 
         } catch (Exception e) {
             e.printStackTrace();
@@ -493,14 +436,10 @@ public class PrismProcessResumeTest extends BaseTestClass {
 
             Util.shutDownService(cluster2.getProcessHelper());
 
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getProcessData()));
-            AssertUtil.assertFailed(prism.getProcessHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[0].getProcessData()));
+            AssertUtil.assertFailed(prism.getProcessHelper().resume(bundles[1].getProcessData()));
             AssertUtil.assertFailed(
-                cluster1.getProcessHelper()
-                    .resume(Util.URLS.RESUME_URL, bundles[1].getProcessData())
-            );
+                cluster1.getProcessHelper().resume(bundles[1].getProcessData()));
         } catch (Exception e) {
             e.printStackTrace();
             throw new TestNGException(e.getCause());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
index 5590c54..e0c25d0 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
@@ -30,7 +30,6 @@ import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
@@ -125,10 +124,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, process1));
-        AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, process2));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(process1));
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().schedule(process2));
 
         //now check if they have been scheduled correctly or not
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
@@ -151,25 +148,25 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcess();
 
         //suspend process on colo-1
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().suspend(URLS.SUSPEND_URL, process1));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().suspend(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //now check if it has been scheduled correctly or not
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(URLS.RESUME_URL, process1));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         //suspend process on colo-2
-        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(URLS.SUSPEND_URL, process2));
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         //now check if it has been scheduled correctly or not
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process2));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(URLS.RESUME_URL, process2));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
     }
 
@@ -185,16 +182,16 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         bundles[0].submitAndScheduleProcess();
         bundles[1].submitAndScheduleProcess();
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process2));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(process2));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
 
-        AssertUtil.assertFailed(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
-        AssertUtil.assertFailed(cluster1.getProcessHelper().schedule(URLS.SCHEDULE_URL, process2));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().schedule(process1));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().schedule(process2));
     }
 
     /**
@@ -204,8 +201,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
      */
     @Test(groups = {"prism", "0.2", "embedded"})
     public void testScheduleNonExistentProcessOnBothColos() throws Exception {
-        AssertUtil.assertFailed(cluster2.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
-        AssertUtil.assertFailed(cluster1.getProcessHelper().schedule(URLS.SCHEDULE_URL, process2));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().schedule(process1));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().schedule(process2));
     }
 
     /**
@@ -220,8 +217,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         try {
             bundles[1].submitProcess(true);
             Util.shutDownService(cluster2.getProcessHelper());
-            AssertUtil.assertSucceeded(prism.getProcessHelper()
-                .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2));
+            AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process2));
 
             //now check if they have been scheduled correctly or not
             AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
@@ -248,7 +244,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         try {
             bundles[0].submitProcess(true);
             Util.shutDownService(cluster2.getProcessHelper());
-            AssertUtil.assertFailed(prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, process1));
+            AssertUtil.assertFailed(prism.getProcessHelper().schedule(process1));
             AssertUtil
                 .checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         } catch (Exception e) {
@@ -271,8 +267,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         throws Exception {
         try {
             bundles[0].submitAndScheduleProcess();
-            AssertUtil.assertSucceeded(cluster1.getProcessHelper()
-                .suspend(URLS.SUSPEND_URL, process1));
+            AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(process1));
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
 
@@ -302,7 +297,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
         throws Exception {
         try {
             bundles[0].submitAndScheduleProcess();
-            AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process1));
+            AssertUtil.assertSucceeded(prism.getProcessHelper().delete(process1));
             AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
 
             bundles[1].submitAndScheduleProcess();
@@ -344,7 +339,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
                 Util.readEntityName(process),Job.Status.KILLED);
             String oldBundleID = InstanceUtil.getLatestBundleID(cluster1,
                 Util.readEntityName(process), EntityType.PROCESS);
-            prism.getProcessHelper().delete(URLS.DELETE_URL, process);
+            prism.getProcessHelper().delete(process);
 
             bundles[0].submitAndScheduleProcess();
             OozieUtil.verifyNewBundleCreation(cluster1, oldBundleID,
@@ -368,9 +363,9 @@ public class PrismProcessScheduleTest extends BaseTestClass {
          ProcessMerlin processObj = new ProcessMerlin(bundles[0].getProcessData());
          processObj.setInputs(null);
          processObj.setLateProcess(null);
-         AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, bundles[0].getOutputFeedFromBundle()));
-         AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                 processObj.toString()));
+         AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(bundles[0].getOutputFeedFromBundle()));
+         AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(
+             processObj.toString()));
          InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(processObj.toString()), 2,
                  CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS, 10);
      }
@@ -388,8 +383,8 @@ public class PrismProcessScheduleTest extends BaseTestClass {
          processObj.setInputs(null);
          processObj.setOutputs(null);
          processObj.setLateProcess(null);
-         AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
-                 processObj.toString()));
+         AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(
+             processObj.toString()));
          InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(processObj.toString()), 2,
                  CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS, 10);
      }


[33/41] git commit: FALCON-690: kinit with realm in order to support deployments with cross domain trust. Contributed by Arpit Gupta

Posted by ra...@apache.org.
FALCON-690: kinit with realm in order to support deployments with cross domain trust. Contributed by Arpit Gupta


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

Branch: refs/heads/FALCON-585
Commit: 120c1c04d468e5a2aeec9c69da2d4664c6795d87
Parents: 581e592
Author: Raghav Kumar Gautam <ra...@apache.org>
Authored: Sat Sep 6 16:38:53 2014 -0700
Committer: Raghav Kumar Gautam <ra...@apache.org>
Committed: Sat Sep 6 16:38:53 2014 -0700

----------------------------------------------------------------------
 .../falcon/regression/core/enumsAndConstants/MerlinConstants.java | 1 +
 .../org/apache/falcon/regression/core/util/KerberosHelper.java    | 3 ++-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/120c1c04/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
index edd75ed..3a56530 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
@@ -46,6 +46,7 @@ public final class MerlinConstants {
     private static HashMap<String, String> keyTabMap;
     public static final String ACL_OWNER = Config.getProperty("ACL.OWNER", RequestKeys.CURRENT_USER);
     public static final String ACL_GROUP = Config.getProperty("ACL.GROUP", "default");
+    public static final String USER_REALM = Config.getProperty("USER.REALM", "");
 
     public static final boolean CLEAN_TEST_DIR =
         Boolean.valueOf(Config.getProperty("clean_test_dir", "true"));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/120c1c04/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/KerberosHelper.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/KerberosHelper.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/KerberosHelper.java
index 71d728e..3c666e7 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/KerberosHelper.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/KerberosHelper.java
@@ -41,7 +41,8 @@ public final class KerberosHelper {
             user = MerlinConstants.CURRENT_USER_NAME;
         }
         final String keytab = MerlinConstants.getKeytabForUser(user);
-        final String command = String.format("kinit -kt %s %s", keytab, user);
+        String principal = MerlinConstants.USER_REALM.isEmpty() ? user : user + '@' + MerlinConstants.USER_REALM;
+        final String command = String.format("kinit -kt %s %s", keytab, principal);
         final int exitVal = ExecUtil.executeCommandGetExitCode(command);
         Assert.assertEquals(exitVal, 0, "Switching Kerberos credential did not succeed.");
     }


[08/41] git commit: Reverse : FALCON-633 RetryTests and Retentions tests should stop using root dir

Posted by ra...@apache.org.
Reverse : FALCON-633 RetryTests and Retentions tests should stop using root dir


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

Branch: refs/heads/FALCON-585
Commit: b1c13df454a46c3df0f6b1041147694721b770d7
Parents: ffe18b0
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Thu Aug 28 13:34:04 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Thu Aug 28 13:34:04 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  6 +-
 .../falcon/regression/core/util/BundleUtil.java |  9 ++-
 .../apache/falcon/regression/NewRetryTest.java  | 79 +++++++++-----------
 .../falcon/regression/prism/RetentionTest.java  |  2 +-
 4 files changed, 45 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1c13df4/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 7a3b84f..52e697e 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -7,11 +7,9 @@ Trunk (Unreleased)
   NEW FEATURES
    FALCON-589 Add test cases for various feed operations on Hcat feeds (Karishma G 
    via Samarth Gupta)
-  IMPROVEMENTS
-
-   FALCON-633 RetryTests and Retentions tests should stop using root dir
-   (Raghav Kumar Gautam via Samarth Gupta)
 
+  IMPROVEMENTS
+   
    FALCON-632 Refactoring, documentation stuff (Paul Isaychuk via Samarth Gupta)
 
    FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1c13df4/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
index d5790c4..1f73523 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
@@ -48,13 +48,14 @@ public final class BundleUtil {
         return readBundleFromFolder("LateDataBundles");
     }
 
-    public static Bundle readRetryBundle(String appPath, String testName) throws IOException {
-        return generateBundleFromTemplate("RetryTests", appPath, testName);
+    public static Bundle readRetryBundle() throws IOException {
+        return readBundleFromFolder("RetryTests");
     }
 
-    public static Bundle readRetentionBundle(String appPath, String testName) throws IOException {
-        return generateBundleFromTemplate("RetentionBundles", appPath, testName);
+    public static Bundle readRetentionBundle() throws IOException {
+        return readBundleFromFolder("RetentionBundles");
     }
+
     public static Bundle readELBundle() throws IOException {
         return readBundleFromFolder("ELbundle");
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1c13df4/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
index 8bcc797..5ab3dfe 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
@@ -74,11 +74,8 @@ public class NewRetryTest extends BaseTestClass {
     DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy/MM/dd/HH/mm");
     final private String baseTestDir = baseHDFSDir + "/NewRetryTest";
     final private String aggregateWorkflowDir = baseTestDir + "/aggregator";
-    final private String lateInputDir = baseTestDir + "/lateDataTest/inputFolders/";
-    final private String lateInputPath = lateInputDir + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-    final private String lateOutputDir = baseTestDir + "/lateDataTest/outputFolders/";
-    final private String lateOutputPath = lateOutputDir
-        + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+    final private String lateDir = baseTestDir + "/lateDataTest/testFolders/";
+    final private String latePath = lateDir + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
     private DateTime startDate;
     private DateTime endDate;
 
@@ -89,17 +86,15 @@ public class NewRetryTest extends BaseTestClass {
 
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
-        bundles[0] = new Bundle(
-            BundleUtil.readRetryBundle(baseAppHDFSDir, this.getClass().getSimpleName()), cluster);
+        bundles[0] = new Bundle(BundleUtil.readRetryBundle(), cluster);
         bundles[0].generateUniqueBundle();
         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         startDate = new DateTime(DateTimeZone.UTC).plusMinutes(1);
         endDate = new DateTime(DateTimeZone.UTC).plusMinutes(2);
         bundles[0].setProcessValidity(startDate, endDate);
 
-        bundles[0].setOutputFeedLocationData(lateOutputPath);
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
         feed = Util.insertLateFeedValue(feed, new Frequency("minutes(8)"));
         bundles[0].getDataSets().remove(bundles[0].getInputFeedFromBundle());
         bundles[0].getDataSets().add(feed);
@@ -131,8 +126,8 @@ public class NewRetryTest extends BaseTestClass {
         } else {
             AssertUtil.assertSucceeded(response);
             // lets create data now:
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 
             //schedule process
             AssertUtil.assertSucceeded(
@@ -182,8 +177,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             //now wait till the process is over
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
@@ -240,8 +235,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
@@ -293,8 +288,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
 
@@ -349,8 +344,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -397,8 +392,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -449,8 +444,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -503,8 +498,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -555,8 +550,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -594,8 +589,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
 
@@ -648,8 +643,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over
@@ -685,7 +680,7 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInSuspendedAndResumeCaseWithLateData(Retry retry) throws Exception {
 
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
         feed = Util.insertLateFeedValue(feed, new Frequency("minutes(10)"));
         bundles[0].getDataSets().remove(bundles[0].getInputFeedFromBundle());
         bundles[0].getDataSets().add(feed);
@@ -703,8 +698,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -779,7 +774,7 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInLateDataCase(Retry retry) throws Exception {
 
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
 
         feed = Util.insertLateFeedValue(feed, getFrequency(retry));
 
@@ -800,11 +795,11 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             List<String> initialData =
                 Util.getHadoopDataFromDir(clusterFS, bundles[0].getInputFeedFromBundle(),
-                    lateInputDir);
+                    lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             String bundleId = OozieUtil.getBundles(clusterOC,
@@ -842,7 +837,7 @@ public class NewRetryTest extends BaseTestClass {
             String insertionFolder =
                 Util.findFolderBetweenGivenTimeStamps(now, now.plusMinutes(5), initialData);
             logger.info("inserting data in folder " + insertionFolder + " at " + DateTime.now());
-            HadoopUtil.injectMoreData(clusterFS, lateInputDir + insertionFolder,
+            HadoopUtil.injectMoreData(clusterFS, lateDir + insertionFolder,
                     OSUtil.OOZIE_EXAMPLE_INPUT_DATA + "lateData");
             //now to validate all failed instances to check if they were retried or not.
             validateRetry(clusterOC, bundleId,
@@ -859,7 +854,7 @@ public class NewRetryTest extends BaseTestClass {
     public void testRetryInDeleteAfterPartialRetryCase(Retry retry) throws Exception {
 
         String feed =
-            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), lateInputPath);
+            Util.setFeedPathValue(bundles[0].getInputFeedFromBundle(), latePath);
         feed = Util.insertLateFeedValue(feed, new Frequency("minutes(1)"));
         bundles[0].getDataSets().remove(bundles[0].getInputFeedFromBundle());
         bundles[0].getDataSets().add(feed);
@@ -878,8 +873,8 @@ public class NewRetryTest extends BaseTestClass {
             AssertUtil.assertFailed(response);
         } else {
             AssertUtil.assertSucceeded(response);
-            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
-            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
+            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
+            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
             AssertUtil.assertSucceeded(
                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
             //now wait till the process is over

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1c13df4/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
index b288b77..1d900d9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
@@ -72,7 +72,7 @@ public class RetentionTest extends BaseTestClass {
     @BeforeMethod(alwaysRun = true)
     public void testName(Method method) throws Exception {
         logger.info("test name: " + method.getName());
-        Bundle bundle = BundleUtil.readRetentionBundle(baseAppHDFSDir, this.getClass().getSimpleName());
+        Bundle bundle = BundleUtil.readRetentionBundle();
         bundles[0] = new Bundle(bundle, cluster);
         bundles[0].setInputFeedDataPath(testHDFSDir);
         bundles[0].generateUniqueBundle();


[16/41] git commit: FALCON-594 Process lineage information for Retention policies. Contributed by Sowmya Ramesh

Posted by ra...@apache.org.
FALCON-594 Process lineage information for Retention policies. Contributed by Sowmya Ramesh


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

Branch: refs/heads/FALCON-585
Commit: d2e5f8c9ee8c97a0a6011261f1cbabf2dbf4f309
Parents: d74dc32
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Tue Sep 2 12:31:25 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Tue Sep 2 12:31:25 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../InstanceRelationshipGraphBuilder.java       |  41 +++++++
 .../falcon/metadata/MetadataMappingService.java |   4 +-
 .../falcon/metadata/RelationshipLabel.java      |   5 +-
 .../apache/falcon/retention/EvictionHelper.java |  88 +++++++++++++++
 .../metadata/MetadataMappingServiceTest.java    | 112 ++++++++++++++++---
 .../apache/falcon/retention/FeedEvictor.java    |  30 +----
 .../falcon/cluster/util/EmbeddedCluster.java    |  24 ++--
 8 files changed, 256 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 075fe7e..7145ff2 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,9 @@ Trunk (Unreleased)
    FALCON-263 API to get workflow parameters. (pavan kumar kolamuri via Shwetha GS)
 
   IMPROVEMENTS
+   FALCON-594 Process lineage information for Retention policies
+   (Sowmya Ramesh via Venkatesh Seetharam)
+
    FALCON-325 Process lineage information for Replication policies
    (Sowmya Ramesh via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java b/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
index 452872e..e7670da 100644
--- a/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
+++ b/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
@@ -20,8 +20,10 @@ package org.apache.falcon.metadata;
 
 import com.tinkerpop.blueprints.Graph;
 import com.tinkerpop.blueprints.Vertex;
+import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.CatalogStorage;
+import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.FeedHelper;
 import org.apache.falcon.entity.Storage;
 import org.apache.falcon.entity.common.FeedDataPath;
@@ -32,6 +34,8 @@ import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.feed.LocationType;
 import org.apache.falcon.entity.v0.process.Process;
+import org.apache.falcon.retention.EvictionHelper;
+import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.falcon.workflow.WorkflowExecutionArgs;
@@ -202,6 +206,43 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
                 RelationshipLabel.FEED_CLUSTER_REPLICATED_EDGE, context.getTimeStampAsISO8601());
     }
 
+    public void addEvictedInstance(WorkflowExecutionContext context) throws FalconException {
+        String outputFeedNamesArg = context.getOutputFeedNames();
+        if ("NONE".equals(outputFeedNamesArg)) {
+            LOG.info("There are no output feeds for this process, return");
+            return;
+        }
+
+        String logFile = context.getLogFile();
+        if (StringUtils.isEmpty(logFile)){
+            throw new IllegalArgumentException("csv log file path empty");
+        }
+
+        String clusterName = context.getClusterName();
+        String[] paths = EvictionHelper.getInstancePaths(ClusterHelper.getFileSystem(clusterName), new Path(logFile));
+        if (paths == null || paths.length <= 0) {
+            throw new IllegalArgumentException("No instance paths in log file");
+        }
+
+        // For retention there will be only one output feed name
+        String feedName = outputFeedNamesArg;
+        for (String feedInstanceDataPath : paths) {
+            LOG.info("Computing feed instance for : name=" + feedName + ", path= "
+                    + feedInstanceDataPath + ", in cluster: " + clusterName);
+            RelationshipType vertexType = RelationshipType.FEED_INSTANCE;
+            String feedInstanceName = getFeedInstanceName(feedName, clusterName, feedInstanceDataPath);
+            Vertex feedInstanceVertex = findVertex(feedInstanceName, vertexType);
+
+            LOG.info("Vertex exists? name={}, type={}, v={}", feedInstanceName, vertexType, feedInstanceVertex);
+            if (feedInstanceVertex == null) {
+                throw new IllegalStateException(vertexType + " instance vertex must exist " + feedInstanceName);
+            }
+
+            addInstanceToEntity(feedInstanceVertex, clusterName, RelationshipType.CLUSTER_ENTITY,
+                    RelationshipLabel.FEED_CLUSTER_EVICTED_EDGE, context.getTimeStampAsISO8601());
+        }
+    }
+
     private void addFeedInstance(Vertex processInstance, RelationshipLabel edgeLabel,
                                  WorkflowExecutionContext context, String feedName,
                                  String feedInstanceDataPath) throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
index ab82ce1..f607e0a 100644
--- a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
+++ b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
@@ -293,8 +293,8 @@ public class MetadataMappingService
         instanceGraphBuilder.addReplicatedInstance(context);
     }
 
-    private void onFeedInstanceEvicted(WorkflowExecutionContext context) {
+    private void onFeedInstanceEvicted(WorkflowExecutionContext context) throws FalconException {
         LOG.info("Adding evicted feed instance: {}", context.getNominalTimeAsISO8601());
-        // todo - tbd
+        instanceGraphBuilder.addEvictedInstance(context);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java b/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
index acd764f..5b312da 100644
--- a/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
+++ b/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
@@ -39,7 +39,10 @@ public enum RelationshipLabel {
     PIPELINES("pipeline"),
 
     // replication labels
-    FEED_CLUSTER_REPLICATED_EDGE("replicated-to");
+    FEED_CLUSTER_REPLICATED_EDGE("replicated-to"),
+
+    // eviction labels
+    FEED_CLUSTER_EVICTED_EDGE("evicted-from");
 
     private final String name;
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/common/src/main/java/org/apache/falcon/retention/EvictionHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/retention/EvictionHelper.java b/common/src/main/java/org/apache/falcon/retention/EvictionHelper.java
new file mode 100644
index 0000000..5d6481c
--- /dev/null
+++ b/common/src/main/java/org/apache/falcon/retention/EvictionHelper.java
@@ -0,0 +1,88 @@
+/**
+ * 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.retention;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.falcon.FalconException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Helper methods to facilitate eviction.
+ */
+
+public final class EvictionHelper {
+
+    private static final Logger LOG = LoggerFactory.getLogger(EvictionHelper.class);
+
+    private static final String INSTANCEPATH_FORMAT = "instancePaths=";
+    public static final String INSTANCEPATH_SEPARATOR = ",";
+
+
+    private EvictionHelper() {}
+
+    public static void logInstancePaths(final FileSystem logfs, final Path path,
+                                        final String data) throws IOException {
+        LOG.info("Writing deleted instances to path {}", path);
+        OutputStream out = logfs.create(path);
+        out.write(INSTANCEPATH_FORMAT.getBytes());
+        out.write(data.getBytes());
+        out.close();
+        debug(logfs, path);
+    }
+
+    public static String[] getInstancePaths(final FileSystem fs,
+                                            final Path logFile) throws FalconException {
+        ByteArrayOutputStream writer = new ByteArrayOutputStream();
+        try {
+            InputStream date = fs.open(logFile);
+            IOUtils.copyBytes(date, writer, 4096, true);
+        } catch (IOException e) {
+            throw new FalconException(e);
+        }
+        String logData = writer.toString();
+        if (StringUtils.isEmpty(logData)) {
+            throw new FalconException("csv file is empty");
+        }
+
+        String[] parts = logData.split(INSTANCEPATH_FORMAT);
+        if (parts.length != 2) {
+            throw new FalconException("Instance path in csv file not in required format: " + logData);
+        }
+
+        // part[0] is instancePaths=
+        return parts[1].split(INSTANCEPATH_SEPARATOR);
+    }
+
+    private static void debug(final FileSystem fs, final Path outPath) throws IOException {
+        ByteArrayOutputStream writer = new ByteArrayOutputStream();
+        InputStream instance = fs.open(outPath);
+        IOUtils.copyBytes(instance, writer, 4096, true);
+        LOG.debug("Instance Paths copied to {}", outPath);
+        LOG.debug("Written {}", writer);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
index 3f3f539..f49ada0 100644
--- a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
@@ -23,6 +23,7 @@ import com.tinkerpop.blueprints.Edge;
 import com.tinkerpop.blueprints.Graph;
 import com.tinkerpop.blueprints.GraphQuery;
 import com.tinkerpop.blueprints.Vertex;
+import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.cluster.util.EntityBuilderTestUtil;
 import org.apache.falcon.entity.Storage;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -39,6 +40,7 @@ import org.apache.falcon.entity.v0.process.Inputs;
 import org.apache.falcon.entity.v0.process.Output;
 import org.apache.falcon.entity.v0.process.Outputs;
 import org.apache.falcon.entity.v0.process.Process;
+import org.apache.falcon.retention.EvictionHelper;
 import org.apache.falcon.security.CurrentUser;
 import org.apache.falcon.service.Services;
 import org.apache.falcon.util.StartupProperties;
@@ -46,6 +48,7 @@ import org.apache.falcon.workflow.WorkflowExecutionArgs;
 import org.apache.falcon.workflow.WorkflowExecutionContext;
 import static org.apache.falcon.workflow.WorkflowExecutionContext.EntityOperations;
 import org.apache.falcon.workflow.WorkflowJobEndNotificationService;
+import org.apache.hadoop.fs.Path;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
@@ -65,7 +68,8 @@ import java.util.Set;
 public class MetadataMappingServiceTest {
 
     public static final String FALCON_USER = "falcon-user";
-    private static final String LOGS_DIR = "target/log";
+    private static final String LOGS_DIR = "/falcon/staging/feed/logs";
+    private static final String LOG_FILE = "instancePaths-2014-01-01-01-00.csv";
     private static final String NOMINAL_TIME = "2014-01-01-01-00";
 
     public static final String CLUSTER_ENTITY_NAME = "primary-cluster";
@@ -74,6 +78,7 @@ public class MetadataMappingServiceTest {
     public static final String COLO_NAME = "west-coast";
     public static final String GENERATE_WORKFLOW_NAME = "imp-click-join-workflow";
     public static final String REPLICATION_WORKFLOW_NAME = "replication-policy-workflow";
+    private static final String EVICTION_WORKFLOW_NAME = "eviction-policy-workflow";
     public static final String WORKFLOW_VERSION = "1.0.9";
 
     public static final String INPUT_FEED_NAMES = "impression-feed#clicks-feed";
@@ -86,6 +91,8 @@ public class MetadataMappingServiceTest {
         "jail://global:00/falcon/imp-click-join1/20140101,jail://global:00/falcon/imp-click-join2/20140101";
     private static final String REPLICATED_OUTPUT_INSTANCE_PATHS =
             "jail://global:00/falcon/imp-click-join1/20140101";
+    private static final String EVICTED_OUTPUT_INSTANCE_PATHS =
+            "jail://global:00/falcon/imp-click-join1/20140101,jail://global:00/falcon/imp-click-join1/20140102";
 
     public static final String BROKER = "org.apache.activemq.ActiveMQConnectionFactory";
 
@@ -97,6 +104,9 @@ public class MetadataMappingServiceTest {
     private List<Feed> inputFeeds = new ArrayList<Feed>();
     private List<Feed> outputFeeds = new ArrayList<Feed>();
     private Process processEntity;
+    private EmbeddedCluster embeddedCluster;
+    private String hdfsUrl;
+    private static String logFilePath;
 
 
     @BeforeClass
@@ -253,6 +263,40 @@ public class MetadataMappingServiceTest {
         Assert.assertEquals(getEdgesCount(service.getGraph()), 74);
     }
 
+    @Test
+    public void   testLineageForRetention() throws Exception {
+        setupForLineageEviciton();
+        String feedName = "imp-click-join1";
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(
+                        EntityOperations.DELETE, EVICTION_WORKFLOW_NAME,
+                        feedName, "IGNORE", "IGNORE", feedName),
+                WorkflowExecutionContext.Type.POST_PROCESSING);
+
+        service.onSuccess(context);
+
+        debug(service.getGraph());
+        GraphUtils.dump(service.getGraph());
+        List<String> expectedFeeds = Arrays.asList("impression-feed/2014-01-01T00:00Z", "clicks-feed/2014-01-01T00:00Z",
+                "imp-click-join1/2014-01-01T00:00Z", "imp-click-join1/2014-01-02T00:00Z");
+        List<String> secureFeeds = Arrays.asList("impression-feed/2014-01-01T00:00Z",
+                "clicks-feed/2014-01-01T00:00Z");
+        List<String> ownedAndSecureFeeds = Arrays.asList("clicks-feed/2014-01-01T00:00Z",
+                "imp-click-join1/2014-01-01T00:00Z", "imp-click-join1/2014-01-02T00:00Z");
+        verifyLineageGraph(RelationshipType.FEED_INSTANCE.getName(), expectedFeeds, secureFeeds, ownedAndSecureFeeds);
+        String[] paths = EVICTED_OUTPUT_INSTANCE_PATHS.split(EvictionHelper.INSTANCEPATH_SEPARATOR);
+        for (String feedInstanceDataPath : paths) {
+            verifyLineageGraphForReplicationOrEviction(feedName, feedInstanceDataPath, context,
+                    RelationshipLabel.FEED_CLUSTER_EVICTED_EDGE);
+        }
+
+        // No new vertices added
+        Assert.assertEquals(getVerticesCount(service.getGraph()), 23);
+        // +1 =  +2 for evicted-from edge from Feed Instance vertex to cluster.
+        // -1 imp-click-join1 is added twice instead of imp-click-join2 so there is one less edge as there is no
+        // classified-as -> Secure edge.
+        Assert.assertEquals(getEdgesCount(service.getGraph()), 72);
+    }
+
     @Test (dependsOnMethods = "testOnAdd")
     public void testOnChange() throws Exception {
         // shutdown the graph and resurrect for testing
@@ -647,11 +691,20 @@ public class MetadataMappingServiceTest {
     }
 
     private void verifyLineageGraph(String feedType) {
+        List<String> expectedFeeds = Arrays.asList("impression-feed/2014-01-01T00:00Z", "clicks-feed/2014-01-01T00:00Z",
+                "imp-click-join1/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z");
+        List<String> secureFeeds = Arrays.asList("impression-feed/2014-01-01T00:00Z",
+                "clicks-feed/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z");
+        List<String> ownedAndSecureFeeds = Arrays.asList("clicks-feed/2014-01-01T00:00Z",
+                "imp-click-join1/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z");
+        verifyLineageGraph(feedType, expectedFeeds, secureFeeds, ownedAndSecureFeeds);
+    }
+
+    private void verifyLineageGraph(String feedType, List<String> expectedFeeds,
+                                    List<String> secureFeeds, List<String> ownedAndSecureFeeds) {
         // feeds owned by a user
         List<String> feedNamesOwnedByUser = getFeedsOwnedByAUser(feedType);
-        List<String> expected = Arrays.asList("impression-feed/2014-01-01T00:00Z", "clicks-feed/2014-01-01T00:00Z",
-                "imp-click-join1/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z");
-        Assert.assertTrue(feedNamesOwnedByUser.containsAll(expected));
+        Assert.assertTrue(feedNamesOwnedByUser.containsAll(expectedFeeds));
 
         Graph graph = service.getGraph();
 
@@ -666,14 +719,10 @@ public class MetadataMappingServiceTest {
         Assert.assertEquals(vertexById, feedInstanceVertex);
 
         // feeds classified as secure
-        verifyFeedsClassifiedAsSecure(feedType,
-                Arrays.asList("impression-feed/2014-01-01T00:00Z",
-                        "clicks-feed/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z"));
+        verifyFeedsClassifiedAsSecure(feedType, secureFeeds);
 
         // feeds owned by a user and classified as secure
-        verifyFeedsOwnedByUserAndClassification(feedType, "Financial",
-                Arrays.asList("clicks-feed/2014-01-01T00:00Z",
-                        "imp-click-join1/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z"));
+        verifyFeedsOwnedByUserAndClassification(feedType, "Financial", ownedAndSecureFeeds);
     }
 
     private void verifyLineageGraphForReplicationOrEviction(String feedName, String feedInstanceDataPath,
@@ -739,7 +788,8 @@ public class MetadataMappingServiceTest {
             "-" + WorkflowExecutionArgs.BRKR_TTL.getName(), "1000",
 
             "-" + WorkflowExecutionArgs.LOG_DIR.getName(), LOGS_DIR,
-            "-" + WorkflowExecutionArgs.LOG_FILE.getName(), LOGS_DIR + "/log.txt",
+            "-" + WorkflowExecutionArgs.LOG_FILE.getName(),
+            (logFilePath != null ? logFilePath : LOGS_DIR + "/log" + ".txt"),
         };
     }
 
@@ -751,27 +801,30 @@ public class MetadataMappingServiceTest {
         clusterEntity = addClusterEntity(CLUSTER_ENTITY_NAME, COLO_NAME,
                 "classification=production");
 
+        addFeedsAndProcess(clusterEntity);
+    }
+
+    private void addFeedsAndProcess(Cluster cluster) throws Exception  {
         // Add input and output feeds
-        Feed impressionsFeed = addFeedEntity("impression-feed", clusterEntity,
+        Feed impressionsFeed = addFeedEntity("impression-feed", cluster,
                 "classified-as=Secure", "analytics", Storage.TYPE.FILESYSTEM,
                 "/falcon/impression-feed/${YEAR}/${MONTH}/${DAY}");
         inputFeeds.add(impressionsFeed);
-        Feed clicksFeed = addFeedEntity("clicks-feed", clusterEntity,
+        Feed clicksFeed = addFeedEntity("clicks-feed", cluster,
                 "classified-as=Secure,classified-as=Financial", "analytics", Storage.TYPE.FILESYSTEM,
                 "/falcon/clicks-feed/${YEAR}-${MONTH}-${DAY}");
         inputFeeds.add(clicksFeed);
-        Feed join1Feed = addFeedEntity("imp-click-join1", clusterEntity,
+        Feed join1Feed = addFeedEntity("imp-click-join1", cluster,
                 "classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
                 "/falcon/imp-click-join1/${YEAR}${MONTH}${DAY}");
         outputFeeds.add(join1Feed);
-        Feed join2Feed = addFeedEntity("imp-click-join2", clusterEntity,
+        Feed join2Feed = addFeedEntity("imp-click-join2", cluster,
                 "classified-as=Secure,classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
                 "/falcon/imp-click-join2/${YEAR}${MONTH}${DAY}");
         outputFeeds.add(join2Feed);
         processEntity = addProcessEntity(PROCESS_ENTITY_NAME, clusterEntity,
                 "classified-as=Critical", "testPipeline,dataReplication_Pipeline", GENERATE_WORKFLOW_NAME,
                 WORKFLOW_VERSION);
-
     }
 
     private void setupForLineageReplication() throws Exception {
@@ -785,6 +838,33 @@ public class MetadataMappingServiceTest {
         addClusterEntity(BCP_CLUSTER_ENTITY_NAME, "east-coast", "classification=bcp");
     }
 
+    private void setupForLineageEviciton() throws Exception {
+        cleanUp();
+        service.init();
+
+        // Add cluster
+        embeddedCluster = EmbeddedCluster.newCluster(CLUSTER_ENTITY_NAME, true, COLO_NAME,
+                "classification=production");
+        clusterEntity = embeddedCluster.getCluster();
+        configStore.publish(EntityType.CLUSTER, clusterEntity);
+        hdfsUrl = embeddedCluster.getConf().get("fs.default.name");
+
+        addFeedsAndProcess(clusterEntity);
+
+        // GENERATE WF should have run before this to create all instance related vertices
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(
+                        EntityOperations.GENERATE, GENERATE_WORKFLOW_NAME,
+                        "imp-click-join1,imp-click-join1", EVICTED_OUTPUT_INSTANCE_PATHS, null, null),
+                WorkflowExecutionContext.Type.POST_PROCESSING);
+        service.onSuccess(context);
+
+        // Write to csv file
+        String csvData = EVICTED_OUTPUT_INSTANCE_PATHS;
+        logFilePath = hdfsUrl + LOGS_DIR + "/" + LOG_FILE;
+        Path path = new Path(logFilePath);
+        EvictionHelper.logInstancePaths(path.getFileSystem(EmbeddedCluster.newConfiguration()), path, csvData);
+    }
+
     private void cleanUp() throws Exception {
         cleanupGraphStore(service.getGraph());
         cleanupConfigurationStore(configStore);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
----------------------------------------------------------------------
diff --git a/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
index 4de7938..114071f 100644
--- a/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
+++ b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
@@ -48,10 +47,7 @@ import org.slf4j.LoggerFactory;
 
 import javax.servlet.jsp.el.ELException;
 import javax.servlet.jsp.el.ExpressionEvaluator;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.io.PrintStream;
 import java.text.DateFormat;
 import java.text.ParseException;
@@ -107,7 +103,7 @@ public class FeedEvictor extends Configured implements Tool {
     }
 
     private final Map<VARS, String> map = new TreeMap<VARS, String>();
-    private final StringBuffer instancePaths = new StringBuffer("instancePaths=");
+    private final StringBuffer instancePaths = new StringBuffer();
     private final StringBuffer buffer = new StringBuffer();
 
     @Override
@@ -129,7 +125,8 @@ public class FeedEvictor extends Configured implements Tool {
         Storage storage = FeedHelper.createStorage(feedStorageType, feedPattern);
         evict(storage, retentionLimit, timeZone);
 
-        logInstancePaths(new Path(logFile));
+        Path path = new Path(logFile);
+        EvictionHelper.logInstancePaths(path.getFileSystem(getConf()), path, instancePaths.toString());
 
         int len = buffer.length();
         if (len > 0) {
@@ -183,7 +180,7 @@ public class FeedEvictor extends Configured implements Tool {
             deleteInstance(fs, path, feedBasePath);
             Date date = getDate(path, feedPath, dateMask, timeZone);
             buffer.append(dateFormat.format(date)).append(',');
-            instancePaths.append(path).append(",");
+            instancePaths.append(path).append(EvictionHelper.INSTANCEPATH_SEPARATOR);
         }
     }
 
@@ -197,15 +194,6 @@ public class FeedEvictor extends Configured implements Tool {
 
     }
 
-    private void logInstancePaths(Path path) throws IOException {
-        LOG.info("Writing deleted instances to path {}", path);
-        FileSystem logfs = path.getFileSystem(getConf());
-        OutputStream out = logfs.create(path);
-        out.write(instancePaths.toString().getBytes());
-        out.close();
-        debug(logfs, path);
-    }
-
     private Pair<Date, Date> getDateRange(String period) throws ELException {
         Long duration = (Long) EVALUATOR.evaluate("${" + period + "}",
                 Long.class, RESOLVER, RESOLVER);
@@ -330,14 +318,6 @@ public class FeedEvictor extends Configured implements Tool {
         deleteParentIfEmpty(fs, path.getParent(), feedBasePath);
     }
 
-    private void debug(FileSystem fs, Path outPath) throws IOException {
-        ByteArrayOutputStream writer = new ByteArrayOutputStream();
-        InputStream instance = fs.open(outPath);
-        IOUtils.copyBytes(instance, writer, 4096, true);
-        LOG.debug("Instance Paths copied to {}", outPath);
-        LOG.debug("Written {}", writer);
-    }
-
     private CommandLine getCommand(String[] args) throws org.apache.commons.cli.ParseException {
         Options options = new Options();
 
@@ -552,7 +532,7 @@ public class FeedEvictor extends Configured implements Tool {
                 String partitionInfo = partitionToDrop.getValues().toString().replace("," , ";");
                 LOG.info("Deleted partition: " + partitionInfo);
                 buffer.append(partSpec).append(',');
-                instancePaths.append(partitionInfo).append(",");
+                instancePaths.append(partitionInfo).append(EvictionHelper.INSTANCEPATH_SEPARATOR);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d2e5f8c9/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
----------------------------------------------------------------------
diff --git a/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java b/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
index 9512fa8..29c2ec4 100644
--- a/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
+++ b/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
@@ -58,11 +58,11 @@ public class EmbeddedCluster {
     }
 
     public static EmbeddedCluster newCluster(final String name) throws Exception {
-        return createClusterAsUser(name, false);
+        return createClusterAsUser(name, false, null, null);
     }
 
     public static EmbeddedCluster newCluster(final String name, boolean global) throws Exception {
-        return createClusterAsUser(name, global);
+        return createClusterAsUser(name, global, null, null);
     }
 
     public static EmbeddedCluster newCluster(final String name,
@@ -71,18 +71,24 @@ public class EmbeddedCluster {
         return hdfsUser.doAs(new PrivilegedExceptionAction<EmbeddedCluster>() {
             @Override
             public EmbeddedCluster run() throws Exception {
-                return createClusterAsUser(name, false);
+                return createClusterAsUser(name, false, null, null);
             }
         });
     }
 
-    private static EmbeddedCluster createClusterAsUser(String name, boolean global) throws IOException {
+    public static EmbeddedCluster newCluster(final String name, boolean global, final String colo,
+                                             final String tags) throws Exception {
+        return createClusterAsUser(name, global, colo, tags);
+    }
+
+    private static EmbeddedCluster createClusterAsUser(String name, boolean global, String colo,
+                                                       String tags) throws IOException {
         EmbeddedCluster cluster = new EmbeddedCluster();
         cluster.conf.set("fs.default.name", "jail://" + (global ? "global" : name) + ":00");
 
         String hdfsUrl = cluster.conf.get("fs.default.name");
         LOG.info("Cluster Namenode = {}", hdfsUrl);
-        cluster.buildClusterObject(name);
+        cluster.buildClusterObject(name, colo, tags);
         return cluster;
     }
 
@@ -90,10 +96,14 @@ public class EmbeddedCluster {
         return FileSystem.get(conf);
     }
 
-    protected void buildClusterObject(String name) {
+    protected void buildClusterObject(String name, String colo, String tags) {
         clusterEntity = new Cluster();
         clusterEntity.setName(name);
-        clusterEntity.setColo("local");
+        clusterEntity.setColo((colo == null) ? "local" : colo);
+        clusterEntity.setDescription("Embedded cluster: " + name);
+        if (tags != null) {
+            clusterEntity.setTags(tags);
+        }
         clusterEntity.setDescription("Embedded cluster: " + name);
 
         Interfaces interfaces = new Interfaces();


[17/41] git commit: FALCON-579 Lineage breaks if feed.xml doesn't have the date pattern in feed path location. Contributed by Sowmya Ramesh

Posted by ra...@apache.org.
FALCON-579 Lineage breaks if feed.xml doesn't have the date pattern in feed path location. Contributed by Sowmya Ramesh


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

Branch: refs/heads/FALCON-585
Commit: 0bd9c775b6a62aa37f2a32d6d41bbe9e9a982b0e
Parents: d2e5f8c
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Tue Sep 2 13:02:46 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Tue Sep 2 13:02:46 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 +
 .../InstanceRelationshipGraphBuilder.java       | 32 ++++++----
 .../metadata/MetadataMappingServiceTest.java    | 63 +++++++++++++++++++-
 3 files changed, 86 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0bd9c775/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7145ff2..ca12d59 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -72,6 +72,9 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-579 Lineage breaks if feed.xml doesn't have the date pattern in
+   feed path location (Sowmya Ramesh via Venkatesh Seetharam)
+
    FALCON-642 OozieProcessWorkflowBuilderTest test failures. (Shwetha GS)
 
    FALCON-630 late data rerun for process broken in trunk. (Shwetha GS)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0bd9c775/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java b/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
index e7670da..2f9fe8e 100644
--- a/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
+++ b/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
@@ -194,7 +194,8 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
         LOG.info("Computing feed instance for : name=" + feedName + ", path= "
                 + feedInstanceDataPath + ", in cluster: " + srcClusterName);
         RelationshipType vertexType = RelationshipType.FEED_INSTANCE;
-        String feedInstanceName = getFeedInstanceName(feedName, srcClusterName, feedInstanceDataPath);
+        String feedInstanceName = getFeedInstanceName(feedName, srcClusterName,
+                feedInstanceDataPath, context.getNominalTimeAsISO8601());
         Vertex feedInstanceVertex = findVertex(feedInstanceName, vertexType);
 
         LOG.info("Vertex exists? name={}, type={}, v={}", feedInstanceName, vertexType, feedInstanceVertex);
@@ -219,7 +220,8 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
         }
 
         String clusterName = context.getClusterName();
-        String[] paths = EvictionHelper.getInstancePaths(ClusterHelper.getFileSystem(clusterName), new Path(logFile));
+        String[] paths = EvictionHelper.getInstancePaths(
+                ClusterHelper.getFileSystem(clusterName), new Path(logFile));
         if (paths == null || paths.length <= 0) {
             throw new IllegalArgumentException("No instance paths in log file");
         }
@@ -230,12 +232,15 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
             LOG.info("Computing feed instance for : name=" + feedName + ", path= "
                     + feedInstanceDataPath + ", in cluster: " + clusterName);
             RelationshipType vertexType = RelationshipType.FEED_INSTANCE;
-            String feedInstanceName = getFeedInstanceName(feedName, clusterName, feedInstanceDataPath);
+            String feedInstanceName = getFeedInstanceName(feedName, clusterName,
+                    feedInstanceDataPath, context.getNominalTimeAsISO8601());
             Vertex feedInstanceVertex = findVertex(feedInstanceName, vertexType);
 
-            LOG.info("Vertex exists? name={}, type={}, v={}", feedInstanceName, vertexType, feedInstanceVertex);
+            LOG.info("Vertex exists? name={}, type={}, v={}",
+                    feedInstanceName, vertexType, feedInstanceVertex);
             if (feedInstanceVertex == null) {
-                throw new IllegalStateException(vertexType + " instance vertex must exist " + feedInstanceName);
+                throw new IllegalStateException(vertexType
+                        + " instance vertex must exist " + feedInstanceName);
             }
 
             addInstanceToEntity(feedInstanceVertex, clusterName, RelationshipType.CLUSTER_ENTITY,
@@ -249,7 +254,8 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
         String clusterName = context.getClusterName();
         LOG.info("Computing feed instance for : name=" + feedName + ", path= "
                 + feedInstanceDataPath + ", in cluster: " + clusterName);
-        String feedInstanceName = getFeedInstanceName(feedName, clusterName, feedInstanceDataPath);
+        String feedInstanceName = getFeedInstanceName(feedName, clusterName,
+                feedInstanceDataPath, context.getNominalTimeAsISO8601());
         LOG.info("Adding feed instance: " + feedInstanceName);
         Vertex feedInstance = addVertex(feedInstanceName, RelationshipType.FEED_INSTANCE,
                 context.getTimeStampAsISO8601());
@@ -271,7 +277,8 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
     }
 
     public static String getFeedInstanceName(String feedName, String clusterName,
-                                      String feedInstancePath) throws FalconException {
+                                             String feedInstancePath,
+                                             String nominalTime) throws FalconException {
         try {
             Feed feed = ConfigurationStore.get().get(EntityType.FEED, feedName);
             Cluster cluster = ConfigurationStore.get().get(EntityType.CLUSTER, clusterName);
@@ -279,7 +286,7 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
             Storage.TYPE storageType = FeedHelper.getStorageType(feed, cluster);
             return storageType == Storage.TYPE.TABLE
                     ? getTableFeedInstanceName(feed, feedInstancePath, storageType)
-                    : getFileSystemFeedInstanceName(feedInstancePath, feed, cluster);
+                    : getFileSystemFeedInstanceName(feedInstancePath, feed, cluster, nominalTime);
 
         } catch (URISyntaxException e) {
             throw new FalconException(e);
@@ -294,7 +301,8 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
     }
 
     private static String getFileSystemFeedInstanceName(String feedInstancePath, Feed feed,
-                                                 Cluster cluster) throws FalconException {
+                                                        Cluster cluster,
+                                                        String nominalTime) throws FalconException {
         Storage rawStorage = FeedHelper.createStorage(cluster, feed);
         String feedPathTemplate = rawStorage.getUriTemplate(LocationType.DATA);
         String instance = feedInstancePath;
@@ -304,7 +312,9 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
             instance = instance.replaceFirst(element, "");
         }
 
-        return feed.getName() + "/"
-                + SchemaHelper.formatDateUTCToISO8601(instance, FEED_INSTANCE_FORMAT);
+        return StringUtils.isEmpty(instance)
+                ? feed.getName() + "/" + nominalTime
+                : feed.getName() + "/"
+                        + SchemaHelper.formatDateUTCToISO8601(instance, FEED_INSTANCE_FORMAT);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0bd9c775/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
index f49ada0..7b73a91 100644
--- a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
@@ -85,6 +85,9 @@ public class MetadataMappingServiceTest {
     public static final String INPUT_INSTANCE_PATHS =
         "jail://global:00/falcon/impression-feed/2014/01/01,jail://global:00/falcon/impression-feed/2014/01/02"
                 + "#jail://global:00/falcon/clicks-feed/2014-01-01";
+    public static final String INPUT_INSTANCE_PATHS_NO_DATE =
+            "jail://global:00/falcon/impression-feed,jail://global:00/falcon/impression-feed"
+                    + "#jail://global:00/falcon/clicks-feed";
 
     public static final String OUTPUT_FEED_NAMES = "imp-click-join1,imp-click-join2";
     public static final String OUTPUT_INSTANCE_PATHS =
@@ -93,6 +96,8 @@ public class MetadataMappingServiceTest {
             "jail://global:00/falcon/imp-click-join1/20140101";
     private static final String EVICTED_OUTPUT_INSTANCE_PATHS =
             "jail://global:00/falcon/imp-click-join1/20140101,jail://global:00/falcon/imp-click-join1/20140102";
+    public static final String OUTPUT_INSTANCE_PATHS_NO_DATE =
+            "jail://global:00/falcon/imp-click-join1,jail://global:00/falcon/imp-click-join2";
 
     public static final String BROKER = "org.apache.activemq.ActiveMQConnectionFactory";
 
@@ -240,6 +245,31 @@ public class MetadataMappingServiceTest {
     }
 
     @Test
+    public void testLineageForNoDateInFeedPath() throws Exception {
+        setupForNoDateInFeedPath();
+
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(
+                        EntityOperations.GENERATE, GENERATE_WORKFLOW_NAME, null,
+                        OUTPUT_INSTANCE_PATHS_NO_DATE, INPUT_INSTANCE_PATHS_NO_DATE, null),
+                WorkflowExecutionContext.Type.POST_PROCESSING);
+        service.onSuccess(context);
+
+        debug(service.getGraph());
+        GraphUtils.dump(service.getGraph());
+
+        // Verify if instance name has nominal time
+        List<String> feedNamesOwnedByUser = getFeedsOwnedByAUser(RelationshipType.FEED_INSTANCE.getName());
+        List<String> expected = Arrays.asList("impression-feed/2014-01-01T01:00Z", "clicks-feed/2014-01-01T01:00Z",
+                "imp-click-join1/2014-01-01T01:00Z", "imp-click-join2/2014-01-01T01:00Z");
+        Assert.assertTrue(feedNamesOwnedByUser.containsAll(expected));
+
+        // +5 = 1 process, 2 inputs, 2 outputs
+        Assert.assertEquals(getVerticesCount(service.getGraph()), 22);
+        //+34 = +26 for feed instances + 8 for process instance
+        Assert.assertEquals(getEdgesCount(service.getGraph()), 65);
+    }
+
+    @Test
     public void  testLineageForReplication() throws Exception {
         setupForLineageReplication();
 
@@ -729,7 +759,7 @@ public class MetadataMappingServiceTest {
                                                             WorkflowExecutionContext context,
                                                             RelationshipLabel edgeLabel) throws Exception {
         String feedInstanceName = InstanceRelationshipGraphBuilder.getFeedInstanceName(feedName
-                , context.getSrcClusterName(), feedInstanceDataPath);
+                , context.getSrcClusterName(), feedInstanceDataPath, context.getNominalTimeAsISO8601());
         Vertex feedVertex = getEntityVertex(feedInstanceName, RelationshipType.FEED_INSTANCE);
 
         Edge edge = feedVertex.getEdges(Direction.OUT, edgeLabel.getName())
@@ -865,6 +895,37 @@ public class MetadataMappingServiceTest {
         EvictionHelper.logInstancePaths(path.getFileSystem(EmbeddedCluster.newConfiguration()), path, csvData);
     }
 
+    private void setupForNoDateInFeedPath() throws Exception {
+        cleanUp();
+        service.init();
+
+        // Add cluster
+        clusterEntity = addClusterEntity(CLUSTER_ENTITY_NAME, COLO_NAME,
+                "classification=production");
+
+        // Add input and output feeds
+        Feed impressionsFeed = addFeedEntity("impression-feed", clusterEntity,
+                "classified-as=Secure", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/impression-feed");
+        inputFeeds.add(impressionsFeed);
+        Feed clicksFeed = addFeedEntity("clicks-feed", clusterEntity,
+                "classified-as=Secure,classified-as=Financial", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/clicks-feed");
+        inputFeeds.add(clicksFeed);
+        Feed join1Feed = addFeedEntity("imp-click-join1", clusterEntity,
+                "classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
+                "/falcon/imp-click-join1");
+        outputFeeds.add(join1Feed);
+        Feed join2Feed = addFeedEntity("imp-click-join2", clusterEntity,
+                "classified-as=Secure,classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
+                "/falcon/imp-click-join2");
+        outputFeeds.add(join2Feed);
+        processEntity = addProcessEntity(PROCESS_ENTITY_NAME, clusterEntity,
+                "classified-as=Critical", "testPipeline,dataReplication_Pipeline", GENERATE_WORKFLOW_NAME,
+                WORKFLOW_VERSION);
+
+    }
+
     private void cleanUp() throws Exception {
         cleanupGraphStore(service.getGraph());
         cleanupConfigurationStore(configStore);


[25/41] git commit: FALCON-643 Tests with zero-output/input scenario amended to match test case. Contributed by Paul Isaychuk

Posted by ra...@apache.org.
FALCON-643 Tests with zero-output/input scenario amended to match test case. Contributed by Paul Isaychuk


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

Branch: refs/heads/FALCON-585
Commit: d9c115e0e9b55b8a61bdce1c43de88a4e1cc6d0d
Parents: 14b9add
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Thu Sep 4 12:56:47 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Thu Sep 4 12:57:35 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  3 +
 .../falcon/regression/NoOutputProcessTest.java  | 18 +---
 .../regression/prism/PrismFeedUpdateTest.java   | 88 ++++++++++----------
 3 files changed, 53 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d9c115e0/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index b60c23c..0b4714a 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,6 +9,9 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
+   FALCON-643 Tests with zero-output/input scenario amended to match test case (Paul Isaychuk via
+   Ruslan Ostafiychuk)
+
    FALCON-660 7 test classes refactored and few of them documented (Paul Isaychuk via
    Ruslan Ostafiychuk)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d9c115e0/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
index 25456a2..2c30f83 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.falcon.regression;
 
+import org.apache.falcon.regression.Entities.ProcessMerlin;
 import org.apache.falcon.regression.core.bundle.Bundle;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.Frequency.TimeUnit;
@@ -62,26 +63,20 @@ public class NoOutputProcessTest extends BaseTestClass {
 
     @BeforeClass(alwaysRun = true)
     public void createTestData() throws Exception {
-
         logger.info("in @BeforeClass");
         HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-
         Bundle b = BundleUtil.readELBundle();
         b.generateUniqueBundle();
         b = new Bundle(b, cluster);
-
         String startDate = "2010-01-03T00:00Z";
         String endDate = "2010-01-03T03:00Z";
-
         b.setInputFeedDataPath(inputPath);
         String prefix = b.getFeedDataPathPrefix();
         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
-
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
     }
 
-
     @BeforeMethod(alwaysRun = true)
     public void testName(Method method) throws Exception {
         logger.info("test name: " + method.getName());
@@ -92,6 +87,9 @@ public class NoOutputProcessTest extends BaseTestClass {
         bundles[0].setInputFeedDataPath(inputPath);
         bundles[0].setProcessValidity("2010-01-03T02:30Z", "2010-01-03T02:45Z");
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
+        ProcessMerlin process = new ProcessMerlin(bundles[0].getProcessData());
+        process.setOutputs(null);
+        bundles[0].setProcessData(process.toString());
         bundles[0].submitFeedsScheduleProcess(prism);
     }
 
@@ -110,16 +108,12 @@ public class NoOutputProcessTest extends BaseTestClass {
         //wait for all the instances to complete
         InstanceUtil.waitTillInstanceReachState(clusterOC, bundles[0].getProcessName(), 3,
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
-
         Assert.assertEquals(messageConsumer.getReceivedMessages().size(), 3,
             " Message for all the 3 instance not found");
-
         messageConsumer.interrupt();
-
         Util.printMessageData(messageConsumer);
     }
 
-
     @Test(enabled = true, groups = {"singleCluster"})
     public void rm() throws Exception {
         JmsMessageConsumer consumerEntityMsg =
@@ -127,22 +121,18 @@ public class NoOutputProcessTest extends BaseTestClass {
         JmsMessageConsumer consumerProcessMsg =
             new JmsMessageConsumer("FALCON." + bundles[0].getProcessName(),
                 cluster.getClusterHelper().getActiveMQ());
-
         consumerEntityMsg.start();
         consumerProcessMsg.start();
 
         //wait for all the instances to complete
         InstanceUtil.waitTillInstanceReachState(clusterOC, bundles[0].getProcessName(), 3,
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
-
         Assert.assertEquals(consumerEntityMsg.getReceivedMessages().size(), 3,
             " Message for all the 3 instance not found");
         Assert.assertEquals(consumerProcessMsg.getReceivedMessages().size(), 3,
             " Message for all the 3 instance not found");
-
         consumerEntityMsg.interrupt();
         consumerProcessMsg.interrupt();
-
         Util.printMessageData(consumerEntityMsg);
         Util.printMessageData(consumerProcessMsg);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d9c115e0/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
index 89f3686..c0e1617 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
@@ -37,6 +37,7 @@ import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
+import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
@@ -59,7 +60,7 @@ public class PrismFeedUpdateTest extends BaseTestClass {
     ColoHelper cluster1 = servers.get(0);
     ColoHelper cluster2 = servers.get(1);
     FileSystem server1FS = serverFS.get(0);
-    OozieClient OC1 = serverOC.get(0);
+    OozieClient cluster1OC = serverOC.get(0);
     String baseTestDir = baseHDFSDir + "/PrismFeedUpdateTest";
     String aggregateWorkflowDir = baseTestDir + "/aggregator";
     public final String cluster1colo = cluster1.getClusterHelper().getColoName();
@@ -97,23 +98,26 @@ public class PrismFeedUpdateTest extends BaseTestClass {
     public void updateFeedQueue_dependentMultipleProcess_oneProcessZeroInput() throws Exception {
         //cluster1colo and cluster2colo are source. feed01 on cluster1colo target cluster2colo,
         // feed02 on cluster2colo target cluster1colo
+        String cluster1Def = bundles[0].getClusters().get(0);
+        String cluster2Def = bundles[1].getClusters().get(0);
 
-        //get 3 unique bundles
         //set cluster colos
         bundles[0].setCLusterColo(cluster1colo);
-        logger.info("cluster bundles[0]: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
-
+        logger.info("cluster bundles[0]: " + Util.prettyPrintXml(cluster1Def));
         bundles[1].setCLusterColo(cluster2colo);
-        logger.info("cluster bundles[1]: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
+        logger.info("cluster bundles[1]: " + Util.prettyPrintXml(cluster2Def));
 
-        //submit 3 clusters
+        //submit 2 clusters
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
+            cluster1Def));
+        AssertUtil.assertSucceeded(prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
+            cluster2Def));
 
         //get 2 unique feeds
         String feed01 = bundles[0].getInputFeedFromBundle();
         String outputFeed = bundles[0].getOutputFeedFromBundle();
 
-        //set source and target for the 2 feeds
-
+        /* set source and target for the 2 feeds */
         //set clusters to null;
         feed01 = InstanceUtil
             .setFeedCluster(feed01,
@@ -126,68 +130,63 @@ public class PrismFeedUpdateTest extends BaseTestClass {
                 XmlUtil.createRtention("hours(10)", ActionType.DELETE), null,
                 ClusterType.SOURCE, null);
 
-
         //set new feed input data
         feed01 = Util.setFeedPathValue(feed01,
             baseTestDir + "/feed01/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
 
-
         //generate data in both the colos cluster1colo and cluster2colo
         String prefix = InstanceUtil.getFeedPrefix(feed01);
+        String startTime = TimeUtil.getTimeWrtSystemTime(-40);
+        System.out.println("Start time = " + startTime);
         HadoopUtil.deleteDirIfExists(prefix.substring(1), server1FS);
-        HadoopUtil.lateDataReplenish(server1FS, 70, 1, prefix, null);
-
-        String startTime = TimeUtil.getTimeWrtSystemTime(-50);
+        HadoopUtil.lateDataReplenish(server1FS, 80, 20, prefix, null);
 
         //set clusters for feed01
         feed01 = InstanceUtil
             .setFeedCluster(feed01, XmlUtil.createValidity(startTime, "2099-01-01T00:00Z"),
                 XmlUtil.createRtention("hours(10)", ActionType.DELETE),
-                Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE,
-                null);
+                Util.readEntityName(cluster1Def), ClusterType.SOURCE, null);
         feed01 = InstanceUtil
             .setFeedCluster(feed01, XmlUtil.createValidity(startTime, "2099-01-01T00:00Z"),
                 XmlUtil.createRtention("hours(10)", ActionType.DELETE),
-                Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET,
-                null);
+                Util.readEntityName(cluster2Def), ClusterType.TARGET, null);
 
         //set clusters for output feed
         outputFeed = InstanceUtil.setFeedCluster(outputFeed,
             XmlUtil.createValidity(startTime, "2099-01-01T00:00Z"),
             XmlUtil.createRtention("hours(10)", ActionType.DELETE),
-            Util.readEntityName(bundles[0].getClusters().get(0)), ClusterType.SOURCE, null);
+            Util.readEntityName(cluster1Def), ClusterType.SOURCE, null);
         outputFeed = InstanceUtil.setFeedCluster(outputFeed,
             XmlUtil.createValidity(startTime, "2099-01-01T00:00Z"),
             XmlUtil.createRtention("hours(10)", ActionType.DELETE),
-            Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null);
-
+            Util.readEntityName(cluster2Def), ClusterType.TARGET, null);
 
         //submit and schedule feeds
         logger.info("feed01: " + Util.prettyPrintXml(feed01));
         logger.info("outputFeed: " + Util.prettyPrintXml(outputFeed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(Util.URLS
+            .SUBMIT_AND_SCHEDULE_URL, feed01));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(Util.URLS
+            .SUBMIT_AND_SCHEDULE_URL, outputFeed));
 
-        //create 2 process with 2 clusters
-
+        /* create 2 process with 2 clusters */
         //get first process
         String process01 = bundles[0].getProcessData();
 
         //add clusters to process
-
         String processStartTime = TimeUtil.getTimeWrtSystemTime(-11);
         String processEndTime = TimeUtil.getTimeWrtSystemTime(70);
-
-
         process01 = InstanceUtil
             .setProcessCluster(process01, null,
                 XmlUtil.createProcessValidity(startTime, "2099-01-01T00:00Z"));
         process01 = InstanceUtil
-            .setProcessCluster(process01, Util.readEntityName(bundles[0].getClusters().get(0)),
+            .setProcessCluster(process01, Util.readEntityName(cluster1Def),
                 XmlUtil.createProcessValidity(processStartTime, processEndTime));
         process01 = InstanceUtil
-            .setProcessCluster(process01, Util.readEntityName(bundles[1].getClusters().get(0)),
+            .setProcessCluster(process01, Util.readEntityName(cluster2Def),
                 XmlUtil.createProcessValidity(processStartTime, processEndTime));
 
-        //get 2nd process :
+        //get 2nd process
         String process02 = process01;
         process02 = InstanceUtil
             .setProcessName(process02, "zeroInputProcess" + new Random().nextInt());
@@ -197,24 +196,27 @@ public class PrismFeedUpdateTest extends BaseTestClass {
         processMerlin.setProcessFeeds(feed, 0, 0, 1);
         process02 = processMerlin.toString();
 
-
         //submit and schedule both process
         logger.info("process: " + Util.prettyPrintXml(process01));
         logger.info("process: " + Util.prettyPrintXml(process02));
-
-
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS
+            .SUBMIT_AND_SCHEDULE_URL, process01));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS
+            .SUBMIT_AND_SCHEDULE_URL, process02));
         logger.info("Wait till process goes into running ");
+        InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(process01), 1,
+            CoordinatorAction.Status.RUNNING, EntityType.PROCESS, 1);
+        InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(process02), 1,
+            CoordinatorAction.Status.RUNNING, EntityType.PROCESS, 1);
 
         //change feed location path
         outputFeed = Util.setFeedProperty(outputFeed, "queueName", "myQueue");
-
         logger.info("updated feed: " + Util.prettyPrintXml(outputFeed));
 
         //update feed first time
-        prism.getFeedHelper().update(outputFeed, outputFeed);
+        AssertUtil.assertSucceeded(prism.getFeedHelper().update(outputFeed, outputFeed));
     }
 
-
     /**
      * schedules a feed and dependent process. Process start and end are in past
      * Test for bug https://issues.apache.org/jira/browse/FALCON-500
@@ -222,24 +224,25 @@ public class PrismFeedUpdateTest extends BaseTestClass {
     @Test
     public void dependentProcessSucceeded()
         throws Exception {
-        bundles[0].setProcessValidity("2014-06-01T04:00Z","2014-06-01T04:02Z");
+        bundles[0].setProcessValidity("2014-06-01T04:00Z", "2014-06-01T04:02Z");
         bundles[0].submitAndScheduleAllFeeds();
         bundles[0].submitAndScheduleProcess();
 
         InstanceUtil.waitTillInstancesAreCreated(cluster1, bundles[0].getProcessData(), 0);
-        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS, bundles[0].getProcessName(),
+        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS,
+            bundles[0].getProcessName(),
             0, 0);
         InstanceUtil.waitForBundleToReachState(cluster1, bundles[0].getProcessName(),
             Job.Status.SUCCEEDED, 20);
 
         FeedMerlin feed = new FeedMerlin(bundles[0].getDataSets().get(0));
-        feed.addProperty("someProp","someVal");
+        feed.addProperty("someProp", "someVal");
         AssertUtil.assertSucceeded(prism.getFeedHelper().update(feed.toString(), feed.toString()));
         //check for new feed bundle creation
         Assert.assertEquals(OozieUtil.getNumberOfBundle(prism, EntityType.FEED,
-            feed.getName()),2);
+            feed.getName()), 2);
         Assert.assertEquals(OozieUtil.getNumberOfBundle(cluster1, EntityType.PROCESS,
-            bundles[0].getProcessName()),1);
+            bundles[0].getProcessName()), 1);
     }
 
     /**
@@ -256,7 +259,8 @@ public class PrismFeedUpdateTest extends BaseTestClass {
         bundles[0].submitAndScheduleProcess();
 
         InstanceUtil.waitTillInstancesAreCreated(cluster1, bundles[0].getProcessData(), 0);
-        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS, bundles[0].getProcessName(),
+        OozieUtil.createMissingDependencies(cluster1, EntityType.PROCESS,
+            bundles[0].getProcessName(),
             0, 0);
 
         FeedMerlin feed = new FeedMerlin(bundles[0].getDataSets().get(0));
@@ -264,9 +268,9 @@ public class PrismFeedUpdateTest extends BaseTestClass {
         AssertUtil.assertSucceeded(prism.getFeedHelper().update(feed.toString(), feed.toString()));
         //check for new feed bundle creation
         Assert.assertEquals(OozieUtil.getNumberOfBundle(cluster1, EntityType.FEED,
-            feed.getName()),2);
+            feed.getName()), 2);
         Assert.assertEquals(OozieUtil.getNumberOfBundle(cluster1, EntityType.PROCESS,
-            bundles[0].getProcessName()),2);
+            bundles[0].getProcessName()), 2);
     }
 
     @AfterClass(alwaysRun = true)


[28/41] git commit: FALCON-669 Missing optional workflow execution listeners configuration results in NPE. Contributed by Raghav Kumar Gautam

Posted by ra...@apache.org.
FALCON-669 Missing optional workflow execution listeners configuration results in NPE. Contributed by Raghav Kumar Gautam


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

Branch: refs/heads/FALCON-585
Commit: 9aad374ab75ca477ada47c4bd2b0aaacd656602a
Parents: 331ad14
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Thu Sep 4 14:41:06 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Thu Sep 4 14:41:06 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 +++
 .../falcon/workflow/WorkflowJobEndNotificationService.java      | 5 +++++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9aad374a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1abbb67..4a3bbc4 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -77,6 +77,9 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-669 Missing optional workflow execution listeners configuration
+   results in NPE (Raghav Kumar Gautam via Venkatesh Seetharam)
+
    FALCON-644 Falcon message producer masks errors in Post processing
    (Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9aad374a/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
index 67f6c79..fb2d58d 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
@@ -18,6 +18,7 @@
 
 package org.apache.falcon.workflow;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.aspect.GenericAlert;
 import org.apache.falcon.entity.v0.SchemaHelper;
@@ -54,6 +55,10 @@ public class WorkflowJobEndNotificationService implements FalconService {
     public void init() throws FalconException {
         String listenerClassNames = StartupProperties.get().getProperty(
                 "workflow.execution.listeners");
+        if (StringUtils.isEmpty(listenerClassNames)) {
+            return;
+        }
+
         for (String listenerClassName : listenerClassNames.split(",")) {
             listenerClassName = listenerClassName.trim();
             if (listenerClassName.isEmpty()) {


[20/41] git commit: FALCON-640 Add ability to specify sort order for orderBy param in RestAPI. Contributed by Balu Vellanki

Posted by ra...@apache.org.
FALCON-640 Add ability to specify sort order for orderBy param in RestAPI. Contributed by Balu Vellanki


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

Branch: refs/heads/FALCON-585
Commit: 2ed0112e0e2379ff95968dd33b86ebe261cf208b
Parents: 7cfa00d
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Tue Sep 2 13:42:38 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Tue Sep 2 13:42:38 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../java/org/apache/falcon/cli/FalconCLI.java   |  29 +++-
 .../org/apache/falcon/client/FalconClient.java  | 133 ++++++++++---------
 docs/src/site/twiki/FalconCLI.twiki             |  12 +-
 docs/src/site/twiki/restapi/EntityList.twiki    |   1 +
 docs/src/site/twiki/restapi/EntitySummary.twiki |   1 +
 docs/src/site/twiki/restapi/InstanceList.twiki  |   1 +
 docs/src/site/twiki/restapi/InstanceLogs.twiki  |   1 +
 .../site/twiki/restapi/InstanceRunning.twiki    |   1 +
 .../src/site/twiki/restapi/InstanceStatus.twiki |   1 +
 .../falcon/resource/AbstractEntityManager.java  |  39 +++---
 .../resource/AbstractInstanceManager.java       |  49 ++++---
 .../AbstractSchedulableEntityManager.java       |   6 +-
 .../resource/proxy/InstanceManagerProxy.java    |  12 +-
 .../proxy/SchedulableEntityManagerProxy.java    |  36 ++---
 .../falcon/resource/EntityManagerTest.java      |  29 ++--
 .../apache/falcon/resource/InstanceManager.java |  13 +-
 .../resource/SchedulableEntityManager.java      |   8 +-
 .../java/org/apache/falcon/cli/FalconCLIIT.java |  62 +++++++--
 .../falcon/resource/EntityManagerJerseyIT.java  |   5 +-
 20 files changed, 277 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 085fa8e..f554236 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,9 @@ Trunk (Unreleased)
    FALCON-263 API to get workflow parameters. (pavan kumar kolamuri via Shwetha GS)
 
   IMPROVEMENTS
+   FALCON-640 Add ability to specify sort order for orderBy param in RestAPI
+   (Balu Vellanki via Venkatesh Seetharam)
+
    FALCON-594 Process lineage information for Retention policies
    (Sowmya Ramesh via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
index a42da13..d6e3598 100644
--- a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
+++ b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
@@ -86,6 +86,7 @@ public class FalconCLI {
     public static final String FILTER_BY_OPT = "filterBy";
     public static final String TAGS_OPT = "tags";
     public static final String ORDER_BY_OPT = "orderBy";
+    public static final String SORT_ORDER_OPT = "sortOrder";
     public static final String OFFSET_OPT = "offset";
     public static final String NUM_RESULTS_OPT = "numResults";
     public static final String NUM_INSTANCES_OPT = "numInstances";
@@ -229,24 +230,27 @@ public class FalconCLI {
         List<LifeCycle> lifeCycles = getLifeCycle(commandLine.getOptionValue(LIFECYCLE_OPT));
         String filterBy = commandLine.getOptionValue(FILTER_BY_OPT);
         String orderBy = commandLine.getOptionValue(ORDER_BY_OPT);
+        String sortOrder = commandLine.getOptionValue(SORT_ORDER_OPT);
         Integer offset = parseIntegerInput(commandLine.getOptionValue(OFFSET_OPT), 0, "offset");
         Integer numResults = parseIntegerInput(commandLine.getOptionValue(NUM_RESULTS_OPT),
                 FalconClient.DEFAULT_NUM_RESULTS, "numResults");
 
         colo = getColo(colo);
         String instanceAction = "instance";
+        validateSortOrder(sortOrder);
         validateInstanceCommands(optionsList, entity, type, colo);
 
 
         if (optionsList.contains(RUNNING_OPT)) {
             validateOrderBy(orderBy, instanceAction);
             validateFilterBy(filterBy, instanceAction);
-            result = client.getRunningInstances(type, entity, colo, lifeCycles, filterBy, orderBy, offset, numResults);
+            result = client.getRunningInstances(type, entity, colo, lifeCycles, filterBy, orderBy, sortOrder,
+                    offset, numResults);
         } else if (optionsList.contains(STATUS_OPT) || optionsList.contains(LIST_OPT)) {
             validateOrderBy(orderBy, instanceAction);
             validateFilterBy(filterBy, instanceAction);
             result = client.getStatusOfInstances(type, entity, start, end, colo, lifeCycles,
-                    filterBy, orderBy, offset, numResults);
+                    filterBy, orderBy, sortOrder, offset, numResults);
         } else if (optionsList.contains(SUMMARY_OPT)) {
             result = client.getSummaryOfInstances(type, entity, start, end, colo, lifeCycles);
         } else if (optionsList.contains(KILL_OPT)) {
@@ -264,7 +268,7 @@ public class FalconCLI {
             validateOrderBy(orderBy, instanceAction);
             validateFilterBy(filterBy, instanceAction);
             result = client.getLogsOfInstances(type, entity, start, end, colo, runId, lifeCycles,
-                    filterBy, orderBy, offset, numResults);
+                    filterBy, orderBy, sortOrder, offset, numResults);
         } else if (optionsList.contains(PARARMS_OPT)) {
             // start time is the nominal time of instance
             result = client.getParamsOfInstance(type, entity, start, colo, clusters, sourceClusters, lifeCycles);
@@ -340,6 +344,7 @@ public class FalconCLI {
         String end = commandLine.getOptionValue(END_OPT);
         String time = commandLine.getOptionValue(EFFECTIVE_OPT);
         String orderBy = commandLine.getOptionValue(ORDER_BY_OPT);
+        String sortOrder = commandLine.getOptionValue(SORT_ORDER_OPT);
         String filterBy = commandLine.getOptionValue(FILTER_BY_OPT);
         String filterTags = commandLine.getOptionValue(TAGS_OPT);
         String fields = commandLine.getOptionValue(FIELDS_OPT);
@@ -348,6 +353,7 @@ public class FalconCLI {
                 FalconClient.DEFAULT_NUM_RESULTS, "numResults");
         Integer numInstances = parseIntegerInput(commandLine.getOptionValue(NUM_INSTANCES_OPT), 7, "numInstances");
         validateEntityType(entityType);
+        validateSortOrder(sortOrder);
         String entityAction = "entity";
 
         if (optionsList.contains(SUBMIT_OPT)) {
@@ -402,7 +408,7 @@ public class FalconCLI {
             validateOrderBy(orderBy, entityAction);
             validateFilterBy(filterBy, entityAction);
             EntityList entityList = client.getEntityList(entityType, fields, filterBy,
-                    filterTags, orderBy, offset, numResults);
+                    filterTags, orderBy, sortOrder, offset, numResults);
             result = entityList != null ? entityList.toString() : "No entity of type (" + entityType + ") found.";
         }  else if (optionsList.contains(SUMMARY_OPT)) {
             validateCluster(cluster);
@@ -410,7 +416,7 @@ public class FalconCLI {
             validateFilterBy(filterBy, entityAction);
             validateOrderBy(orderBy, entityAction);
             result = client.getEntitySummary(entityType, cluster, start, end, fields, filterBy, filterTags,
-                    orderBy, offset, numResults, numInstances);
+                    orderBy, sortOrder, offset, numResults, numInstances);
         } else if (optionsList.contains(HELP_CMD)) {
             OUT.get().println("Falcon Help");
         } else {
@@ -419,6 +425,15 @@ public class FalconCLI {
         OUT.get().println(result);
     }
 
+    private void validateSortOrder(String sortOrder) throws FalconCLIException {
+        if (!StringUtils.isEmpty(sortOrder)) {
+            if (!sortOrder.equalsIgnoreCase("asc") && !sortOrder.equalsIgnoreCase("desc")) {
+                throw new FalconCLIException("Value for param sortOrder should be \"asc\" or \"desc\". It is  : "
+                        + sortOrder);
+            }
+        }
+    }
+
     private void validateCluster(String cluster) throws FalconCLIException {
         if (StringUtils.isEmpty(cluster)) {
             throw new FalconCLIException("Missing argument: cluster");
@@ -617,6 +632,7 @@ public class FalconCLI {
         Option filterTags = new Option(TAGS_OPT, true, "Filter returned entities by the specified tags");
         Option orderBy = new Option(ORDER_BY_OPT, true,
                 "Order returned entities by this field");
+        Option sortOrder = new Option(SORT_ORDER_OPT, true, "asc or desc order for results");
         Option offset = new Option(OFFSET_OPT, true,
                 "Start returning entities from this offset");
         Option numResults = new Option(NUM_RESULTS_OPT, true,
@@ -638,6 +654,7 @@ public class FalconCLI {
         entityOptions.addOption(filterBy);
         entityOptions.addOption(filterTags);
         entityOptions.addOption(orderBy);
+        entityOptions.addOption(sortOrder);
         entityOptions.addOption(offset);
         entityOptions.addOption(numResults);
         entityOptions.addOption(numInstances);
@@ -747,6 +764,7 @@ public class FalconCLI {
                 "Filter returned instances by the specified fields");
         Option orderBy = new Option(ORDER_BY_OPT, true,
                 "Order returned instances by this field");
+        Option sortOrder = new Option(SORT_ORDER_OPT, true, "asc or desc order for results");
         Option offset = new Option(OFFSET_OPT, true,
                 "Start returning instances from this offset");
         Option numResults = new Option(NUM_RESULTS_OPT, true,
@@ -767,6 +785,7 @@ public class FalconCLI {
         instanceOptions.addOption(filterBy);
         instanceOptions.addOption(offset);
         instanceOptions.addOption(orderBy);
+        instanceOptions.addOption(sortOrder);
         instanceOptions.addOption(numResults);
 
         return instanceOptions;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index 619955b..73635fa 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -338,34 +338,37 @@ public class FalconClient {
     //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
 
     public EntityList getEntityList(String entityType, String fields, String filterBy, String filterTags,
-                                    String orderBy, Integer offset, Integer numResults) throws FalconCLIException {
+                                    String orderBy, String sortOrder,
+                                    Integer offset, Integer numResults) throws FalconCLIException {
         return sendListRequest(Entities.LIST, entityType, fields, filterBy,
-                filterTags, orderBy, offset, numResults);
+                filterTags, orderBy, sortOrder, offset, numResults);
     }
 
     public String getEntitySummary(String entityType, String cluster, String start, String end,
                                    String fields, String filterBy, String filterTags,
-                                   String orderBy, Integer offset, Integer numResults, Integer numInstances)
+                                   String orderBy, String sortOrder,
+                                   Integer offset, Integer numResults, Integer numInstances)
         throws FalconCLIException {
         return sendEntitySummaryRequest(Entities.SUMMARY, entityType, cluster, start, end, fields, filterBy, filterTags,
-                orderBy, offset, numResults, numInstances);
+                orderBy, sortOrder, offset, numResults, numInstances);
     }
 
     public String getRunningInstances(String type, String entity, String colo, List<LifeCycle> lifeCycles,
-                                      String filterBy, String orderBy, Integer offset, Integer numResults)
-        throws FalconCLIException {
+                                      String filterBy, String orderBy, String sortOrder,
+                                      Integer offset, Integer numResults) throws FalconCLIException {
 
         return sendInstanceRequest(Instances.RUNNING, type, entity, null, null,
-                null, null, colo, lifeCycles, filterBy, orderBy, offset, numResults);
+                null, null, colo, lifeCycles, filterBy, orderBy, sortOrder, offset, numResults);
     }
 
     public String getStatusOfInstances(String type, String entity,
                                        String start, String end,
                                        String colo, List<LifeCycle> lifeCycles, String filterBy,
-                                       String orderBy, Integer offset, Integer numResults) throws FalconCLIException {
+                                       String orderBy, String sortOrder,
+                                       Integer offset, Integer numResults) throws FalconCLIException {
 
         return sendInstanceRequest(Instances.STATUS, type, entity, start, end,
-                null, null, colo, lifeCycles, filterBy, orderBy, offset, numResults);
+                null, null, colo, lifeCycles, filterBy, orderBy, sortOrder, offset, numResults);
     }
 
     public String getSummaryOfInstances(String type, String entity,
@@ -440,11 +443,11 @@ public class FalconClient {
     public String getLogsOfInstances(String type, String entity, String start,
                                      String end, String colo, String runId,
                                      List<LifeCycle> lifeCycles, String filterBy,
-                                     String orderBy, Integer offset, Integer numResults)
+                                     String orderBy, String sortOrder, Integer offset, Integer numResults)
         throws FalconCLIException {
 
         return sendInstanceRequest(Instances.LOG, type, entity, start, end,
-                null, runId, colo, lifeCycles, filterBy, orderBy, offset, numResults);
+                null, runId, colo, lifeCycles, filterBy, orderBy, sortOrder, offset, numResults);
     }
 
     public String getParamsOfInstance(String type, String entity,
@@ -533,23 +536,17 @@ public class FalconClient {
         return parseAPIResult(clientResponse);
     }
 
-    private String sendEntitySummaryRequest(Entities entities, String entityType, String cluster,
-                                            String start, String end,
-                                            String fields, String filterBy, String filterTags,
-                                            String orderBy, Integer offset, Integer numResults,
-                                            Integer numInstances) throws FalconCLIException {
-        WebResource resource;
-        if (StringUtils.isEmpty(cluster)) {
-            resource = service.path(entities.path).path(entityType);
-        } else {
-            resource = service.path(entities.path).path(entityType).path(cluster);
-        }
+    private WebResource addParamsToResource(WebResource resource,
+                                            String start, String end, String runId, String colo,
+                                            String fields, String filterBy, String tags,
+                                            String orderBy, String sortOrder,
+                                            Integer offset, Integer numResults, Integer numInstances) {
 
         if (!StringUtils.isEmpty(fields)) {
             resource = resource.queryParam("fields", fields);
         }
-        if (!StringUtils.isEmpty(filterTags)) {
-            resource = resource.queryParam("tags", filterTags);
+        if (!StringUtils.isEmpty(tags)) {
+            resource = resource.queryParam("tags", tags);
         }
         if (!StringUtils.isEmpty(filterBy)) {
             resource = resource.queryParam("filterBy", filterBy);
@@ -557,16 +554,50 @@ public class FalconClient {
         if (!StringUtils.isEmpty(orderBy)) {
             resource = resource.queryParam("orderBy", orderBy);
         }
+        if (!StringUtils.isEmpty(sortOrder)) {
+            resource = resource.queryParam("sortOrder", sortOrder);
+        }
         if (!StringUtils.isEmpty(start)) {
             resource = resource.queryParam("start", start);
         }
         if (!StringUtils.isEmpty(end)) {
             resource = resource.queryParam("end", end);
         }
+        if (runId != null) {
+            resource = resource.queryParam("runid", runId);
+        }
+        if (colo != null) {
+            resource = resource.queryParam("colo", colo);
+        }
+        if (offset != null) {
+            resource = resource.queryParam("offset", offset.toString());
+        }
+        if (numResults != null) {
+            resource = resource.queryParam("numResults", numResults.toString());
+        }
+        if (numInstances != null) {
+            resource = resource.queryParam("numInstances", numInstances.toString());
+        }
+        return resource;
+
+    }
+
+    private String sendEntitySummaryRequest(Entities entities, String entityType, String cluster,
+                                            String start, String end,
+                                            String fields, String filterBy, String filterTags,
+                                            String orderBy, String sortOrder, Integer offset, Integer numResults,
+                                            Integer numInstances) throws FalconCLIException {
+        WebResource resource;
+        if (StringUtils.isEmpty(cluster)) {
+            resource = service.path(entities.path).path(entityType);
+        } else {
+            resource = service.path(entities.path).path(entityType).path(cluster);
+        }
 
-        resource = resource.queryParam("offset", offset.toString());
-        resource = resource.queryParam("numResults", numResults.toString());
-        resource = resource.queryParam("numInstances", numInstances.toString());
+        resource = addParamsToResource(resource, start, end, null, null,
+                fields, filterBy, filterTags,
+                orderBy, sortOrder,
+                offset, numResults, numInstances);
 
         ClientResponse clientResponse = resource
                 .header("Cookie", AUTH_COOKIE_EQ + authenticationToken)
@@ -628,36 +659,19 @@ public class FalconClient {
                                        String runid, String colo,
                                        List<LifeCycle> lifeCycles) throws FalconCLIException {
         return sendInstanceRequest(instances, type, entity, start, end, props,
-                runid, colo, lifeCycles, "", "", 0, DEFAULT_NUM_RESULTS);
+                runid, colo, lifeCycles, "", "", "", 0, DEFAULT_NUM_RESULTS);
     }
 
     private String sendInstanceRequest(Instances instances, String type, String entity,
                                        String start, String end, InputStream props, String runid, String colo,
-                                       List<LifeCycle> lifeCycles, String filterBy,
-                                       String orderBy, Integer offset, Integer numResults) throws FalconCLIException {
+                                       List<LifeCycle> lifeCycles, String filterBy, String orderBy, String sortOrder,
+                                       Integer offset, Integer numResults) throws FalconCLIException {
         checkType(type);
         WebResource resource = service.path(instances.path).path(type)
                 .path(entity);
-        if (start != null) {
-            resource = resource.queryParam("start", start);
-        }
-        if (end != null) {
-            resource = resource.queryParam("end", end);
-        }
-        if (runid != null) {
-            resource = resource.queryParam("runid", runid);
-        }
-        if (colo != null) {
-            resource = resource.queryParam("colo", colo);
-        }
-        if (!StringUtils.isEmpty(filterBy)) {
-            resource = resource.queryParam("filterBy", filterBy);
-        }
-        if (!StringUtils.isEmpty(orderBy)) {
-            resource = resource.queryParam("orderBy", orderBy);
-        }
-        resource = resource.queryParam("offset", offset.toString());
-        resource = resource.queryParam("numResults", numResults.toString());
+
+        resource = addParamsToResource(resource, start, end, runid, colo,
+                null, filterBy, null, orderBy, sortOrder, offset, numResults, null);
 
         if (lifeCycles != null) {
             checkLifeCycleOption(lifeCycles, type);
@@ -717,24 +731,13 @@ public class FalconClient {
 
     //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     private EntityList sendListRequest(Entities entities, String entityType, String fields, String filterBy,
-                                       String filterTags, String orderBy, Integer offset,
+                                       String filterTags, String orderBy, String sortOrder, Integer offset,
                                        Integer numResults) throws FalconCLIException {
         WebResource resource = service.path(entities.path)
                 .path(entityType);
-        if (!StringUtils.isEmpty(filterBy)) {
-            resource = resource.queryParam("filterBy", filterBy);
-        }
-        if (!StringUtils.isEmpty(orderBy)) {
-            resource = resource.queryParam("orderBy", orderBy);
-        }
-        if (!StringUtils.isEmpty(fields)) {
-            resource = resource.queryParam("fields", fields);
-        }
-        if (!StringUtils.isEmpty(filterTags)) {
-            resource = resource.queryParam("tags", filterTags);
-        }
-        resource = resource.queryParam("offset", offset.toString());
-        resource = resource.queryParam("numResults", numResults.toString());
+        resource = addParamsToResource(resource, null, null, null, null, fields, filterBy, filterTags,
+                orderBy, sortOrder, offset, numResults, null);
+
         ClientResponse clientResponse = resource
                 .header("Cookie", AUTH_COOKIE_EQ + authenticationToken)
                 .accept(entities.mimeType).type(MediaType.TEXT_XML)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/FalconCLI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/FalconCLI.twiki b/docs/src/site/twiki/FalconCLI.twiki
index 4ea80c1..3b945e9 100644
--- a/docs/src/site/twiki/FalconCLI.twiki
+++ b/docs/src/site/twiki/FalconCLI.twiki
@@ -52,7 +52,7 @@ Usage:
 $FALCON_HOME/bin/falcon entity -type [cluster|feed|process] -list
 
 Optional Args : -fields <<field1,field2>> -filterBy <<field1:value1,field2:value2>> -tags <<tagkey=tagvalue,tagkey=tagvalue>>
--orderBy <<field>> -offset 0 -numResults 10
+-orderBy <<field>> -sortOrder <<sortOrder>> -offset 0 -numResults 10
 
 <a href="./Restapi/EntityList.html">Optional params described here.</a>
 
@@ -65,7 +65,7 @@ $FALCON_HOME/bin/falcon entity -type [cluster|feed|process] -summary
 
 Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'" -fields <<field1,field2>>
 -filterBy <<field1:value1,field2:value2>> -tags <<tagkey=tagvalue,tagkey=tagvalue>>
--orderBy <<field>> -offset 0 -numResults 10 -numInstances 7
+-orderBy <<field>> -sortOrder <<sortOrder>> -offset 0 -numResults 10 -numInstances 7
 
 <a href="./Restapi/EntitySummary.html">Optional params described here.</a>
 
@@ -157,7 +157,7 @@ $FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -status
 
 Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'" -colo <<colo>>
 -filterBy <<field1:value1,field2:value2>> -lifecycle <<lifecycles>>
--orderBy field -offset 0 -numResults 10
+-orderBy field -sortOrder <<sortOrder>> -offset 0 -numResults 10
 
 <a href="./Restapi/InstanceStatus.html"> Optional params described here.</a>
 
@@ -175,7 +175,7 @@ $FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -list
 
 Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'"
 -colo <<colo>> -lifecycle <<lifecycles>>
--filterBy <<field1:value1,field2:value2>> -orderBy field -offset 0 -numResults 10
+-filterBy <<field1:value1,field2:value2>> -orderBy field -sortOrder <<sortOrder>> -offset 0 -numResults 10
 
 <a href="./Restapi/InstanceList.html">Optional params described here.</a>
 
@@ -205,7 +205,7 @@ Usage:
 $FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -running
 
 Optional Args : -colo <<colo>> -lifecycle <<lifecycles>>
--filterBy <<field1:value1,field2:value2>> -orderBy <<field>> -offset 0 -numResults 10
+-filterBy <<field1:value1,field2:value2>> -orderBy <<field>> -sortOrder <<sortOrder>> -offset 0 -numResults 10
 
 <a href="./Restapi/InstanceRunning.html">Optional params described here.</a>
 
@@ -218,7 +218,7 @@ $FALCON_HOME/bin/falcon instance -type <<feed/process>> -name <<name>> -logs
 
 Optional Args : -start "yyyy-MM-dd'T'HH:mm'Z'" -end "yyyy-MM-dd'T'HH:mm'Z'" -runid <<runid>>
 -colo <<colo>> -lifecycle <<lifecycles>>
--filterBy <<field1:value1,field2:value2>> -orderBy field -offset 0 -numResults 10
+-filterBy <<field1:value1,field2:value2>> -orderBy field -sortOrder <<sortOrder>> -offset 0 -numResults 10
 
 <a href="./Restapi/InstanceLogs.html">Optional params described here.</a>
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/restapi/EntityList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/EntityList.twiki b/docs/src/site/twiki/restapi/EntityList.twiki
index 353007c..b569ade 100644
--- a/docs/src/site/twiki/restapi/EntityList.twiki
+++ b/docs/src/site/twiki/restapi/EntityList.twiki
@@ -18,6 +18,7 @@ Get list of the entities.
       * Example: tags=consumer=consumer@xyz.com,owner=producer@xyz.com
    * orderBy <optional param> Field by which results should be ordered.
       * Supports ordering by "name".
+   * sortOrder <optional param> Valid options are "asc" and "desc"
    * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
    * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/restapi/EntitySummary.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/EntitySummary.twiki b/docs/src/site/twiki/restapi/EntitySummary.twiki
index 6e6ddf4..f002d81 100644
--- a/docs/src/site/twiki/restapi/EntitySummary.twiki
+++ b/docs/src/site/twiki/restapi/EntitySummary.twiki
@@ -23,6 +23,7 @@ Given an EntityType and cluster, get list of entities along with summary of N re
       * Example: tags=consumer=consumer@xyz.com,owner=producer@xyz.com
    * orderBy <optional param> Field by which results should be ordered.
       * Supports ordering by "name".
+   * sortOrder <optional param> Valid options are "asc" and "desc"
    * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
    * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
    * numInstances <optional param> Number of recent instances to show per entity. Only integers > 0 are valid, Default is 7.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/restapi/InstanceList.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceList.twiki b/docs/src/site/twiki/restapi/InstanceList.twiki
index 76eaa33..2cd9b1c 100644
--- a/docs/src/site/twiki/restapi/InstanceList.twiki
+++ b/docs/src/site/twiki/restapi/InstanceList.twiki
@@ -21,6 +21,7 @@ Get list of all instances of a given entity.
       * Query will do an AND among filterBy fields.
    * orderBy <optional param> Field by which results should be ordered.
       * Supports ordering by  "status","startTime","endTime","cluster".
+   * sortOrder <optional param> Valid options are "asc" and "desc"
    * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
    * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
    

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/restapi/InstanceLogs.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceLogs.twiki b/docs/src/site/twiki/restapi/InstanceLogs.twiki
index 354eed4..c1103b7 100644
--- a/docs/src/site/twiki/restapi/InstanceLogs.twiki
+++ b/docs/src/site/twiki/restapi/InstanceLogs.twiki
@@ -22,6 +22,7 @@ Get log of a specific instance of an entity.
       * Query will do an AND among filterBy fields.
    * orderBy <optional param> Field by which results should be ordered.
       * Supports ordering by "status","startTime","endTime","cluster".
+   * sortOrder <optional param> Valid options are "asc" and "desc"
    * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
    * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/restapi/InstanceRunning.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceRunning.twiki b/docs/src/site/twiki/restapi/InstanceRunning.twiki
index 47037d6..dcd2230 100644
--- a/docs/src/site/twiki/restapi/InstanceRunning.twiki
+++ b/docs/src/site/twiki/restapi/InstanceRunning.twiki
@@ -17,6 +17,7 @@ Get a list of instances currently running for a given entity.
       * Query will do an AND among filterBy fields.
    * orderBy <optional param> Field by which results should be ordered.
       * Supports ordering by "status","startTime","endTime","cluster".
+   * sortOrder <optional param> Valid options are "asc" and "desc"
    * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
    * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/docs/src/site/twiki/restapi/InstanceStatus.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/restapi/InstanceStatus.twiki b/docs/src/site/twiki/restapi/InstanceStatus.twiki
index 69634de..ece8c3f 100644
--- a/docs/src/site/twiki/restapi/InstanceStatus.twiki
+++ b/docs/src/site/twiki/restapi/InstanceStatus.twiki
@@ -21,6 +21,7 @@ Get status of a specific instance of an entity.
       * Query will do an AND among filterBy fields.
    * orderBy <optional param> Field by which results should be ordered.
       * Supports ordering by "status","startTime","endTime","cluster".
+   * sortOrder <optional param> Valid options are "asc" and "desc"
    * offset <optional param> Show results from the offset, used for pagination. Defaults to 0.
    * numResults <optional param> Number of results to show per request, used for pagination. Only integers > 0 are valid, Default is 10.
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
index d12dede..fcd7b29 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
@@ -473,12 +473,12 @@ public abstract class AbstractEntityManager {
      * @return EntityList
      */
     public EntityList getEntityList(String type, String fieldStr, String filterBy, String filterTags,
-                                    String orderBy, Integer offset, Integer resultsPerPage) {
+                                    String orderBy, String sortOrder, Integer offset, Integer resultsPerPage) {
 
         HashSet<String> fields = new HashSet<String>(Arrays.asList(fieldStr.toLowerCase().split(",")));
         List<Entity> entities;
         try {
-            entities = getEntities(type, "", "", "", filterBy, filterTags, orderBy, offset, resultsPerPage);
+            entities = getEntities(type, "", "", "", filterBy, filterTags, orderBy, sortOrder, offset, resultsPerPage);
         } catch (Exception e) {
             LOG.error("Failed to get entity list", e);
             throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
@@ -490,7 +490,7 @@ public abstract class AbstractEntityManager {
     }
 
     protected List<Entity> getEntities(String type, String startDate, String endDate, String cluster,
-                                       String filterBy, String filterTags, String orderBy,
+                                       String filterBy, String filterTags, String orderBy, String sortOrder,
                                        int offset, int resultsPerPage) throws FalconException {
         final HashMap<String, String> filterByFieldsValues = getFilterByFieldsValues(filterBy);
         final ArrayList<String> filterByTags = getFilterByTags(filterTags);
@@ -529,7 +529,7 @@ public abstract class AbstractEntityManager {
             entities.add(entity);
         }
         // Sort entities before returning a subset of entity elements.
-        entities = sortEntities(entities, orderBy);
+        entities = sortEntities(entities, orderBy, sortOrder);
 
         int pageCount = getRequiredNumberOfResults(entities.size(), offset, resultsPerPage);
         if (pageCount == 0) {  // handle pagination
@@ -702,27 +702,19 @@ public abstract class AbstractEntityManager {
         return filterEntity;
     }
 
-    private ArrayList<Entity> sortEntities(ArrayList<Entity> entities, String orderBy) {
+    private ArrayList<Entity> sortEntities(ArrayList<Entity> entities, String orderBy, String sortOrder) {
         // Sort the ArrayList using orderBy param
         if (!StringUtils.isEmpty(orderBy)) {
             EntityList.EntityFieldList orderByField = EntityList.EntityFieldList.valueOf(orderBy.toUpperCase());
-
+            final String order = getValidSortOrder(sortOrder, orderBy);
             switch (orderByField) {
 
-            case TYPE:
-                Collections.sort(entities, new Comparator<Entity>() {
-                    @Override
-                    public int compare(Entity e1, Entity e2) {
-                        return e1.getEntityType().compareTo(e2.getEntityType());
-                    }
-                });
-                break;
-
             case NAME:
                 Collections.sort(entities, new Comparator<Entity>() {
                     @Override
                     public int compare(Entity e1, Entity e2) {
-                        return e1.getName().compareTo(e2.getName());
+                        return (order.equalsIgnoreCase("asc")) ? e1.getName().compareTo(e2.getName())
+                                : e2.getName().compareTo(e1.getName());
                     }
                 });
                 break;
@@ -735,6 +727,21 @@ public abstract class AbstractEntityManager {
         return entities;
     }
 
+    protected String getValidSortOrder(String sortOrder, String orderBy) {
+        if (StringUtils.isEmpty(sortOrder)) {
+            return (orderBy.equalsIgnoreCase("starttime")
+                    || orderBy.equalsIgnoreCase("endtime")) ? "desc" : "asc";
+        }
+
+        if (sortOrder.equalsIgnoreCase("asc") || sortOrder.equalsIgnoreCase("desc")) {
+            return sortOrder;
+        }
+
+        String err = "Value for param sortOrder should be \"asc\" or \"desc\". It is  : " + sortOrder;
+        LOG.error(err);
+        throw FalconWebException.newException(err, Response.Status.BAD_REQUEST);
+    }
+
     protected int getRequiredNumberOfResults(int arraySize, int offset, int numresults) {
         /* Get a subset of elements based on offset and count. When returning subset of elements,
               elements[offset] is included. Size 10, offset 10, return empty list.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
index e2c465a..5e351f6 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
@@ -86,7 +86,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
     //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     public InstancesResult getRunningInstances(String type, String entity,
                                                String colo, List<LifeCycle> lifeCycles, String filterBy,
-                                               String orderBy, Integer offset, Integer numResults) {
+                                               String orderBy, String sortOrder, Integer offset, Integer numResults) {
         checkColo(colo);
         checkType(type);
         try {
@@ -95,7 +95,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
             Entity entityObject = EntityUtil.getEntity(type, entity);
             return getInstanceResultSubset(wfEngine.getRunningInstances(entityObject, lifeCycles),
-                    filterBy, orderBy, offset, numResults);
+                    filterBy, orderBy, sortOrder, offset, numResults);
         } catch (Throwable e) {
             LOG.error("Failed to get running instances", e);
             throw FalconWebException.newInstanceException(e, Response.Status.BAD_REQUEST);
@@ -105,14 +105,16 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
     //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     public InstancesResult getInstances(String type, String entity, String startStr, String endStr,
                                         String colo, List<LifeCycle> lifeCycles,
-                                        String filterBy, String orderBy, Integer offset, Integer numResults) {
+                                        String filterBy, String orderBy, String sortOrder,
+                                        Integer offset, Integer numResults) {
         return getStatus(type, entity, startStr, endStr, colo, lifeCycles,
-                filterBy, orderBy, offset, numResults);
+                filterBy, orderBy, sortOrder, offset, numResults);
     }
 
     public InstancesResult getStatus(String type, String entity, String startStr, String endStr,
                                      String colo, List<LifeCycle> lifeCycles,
-                                     String filterBy, String orderBy, Integer offset, Integer numResults) {
+                                     String filterBy, String orderBy, String sortOrder,
+                                     Integer offset, Integer numResults) {
         checkColo(colo);
         checkType(type);
         try {
@@ -125,7 +127,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
             return getInstanceResultSubset(wfEngine.getStatus(entityObject,
                             startAndEndDate.first, startAndEndDate.second, lifeCycles),
-                    filterBy, orderBy, offset, numResults);
+                    filterBy, orderBy, sortOrder, offset, numResults);
         } catch (Throwable e) {
             LOG.error("Failed to get instances status", e);
             throw FalconWebException
@@ -151,16 +153,15 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         }
     }
 
-    public InstancesResult getLogs(String type, String entity, String startStr,
-                                   String endStr, String colo, String runId,
-                                   List<LifeCycle> lifeCycles,
-                                   String filterBy, String orderBy, Integer offset, Integer numResults) {
-
+    public InstancesResult getLogs(String type, String entity, String startStr, String endStr,
+                                   String colo, String runId, List<LifeCycle> lifeCycles,
+                                   String filterBy, String orderBy, String sortOrder,
+                                   Integer offset, Integer numResults) {
         try {
             lifeCycles = checkAndUpdateLifeCycle(lifeCycles, type);
             // getStatus does all validations and filters clusters
             InstancesResult result = getStatus(type, entity, startStr, endStr,
-                    colo, lifeCycles, filterBy, orderBy, offset, numResults);
+                    colo, lifeCycles, filterBy, orderBy, sortOrder, offset, numResults);
             LogProvider logProvider = new LogProvider();
             Entity entityObject = EntityUtil.getEntity(type, entity);
             for (Instance instance : result.getInstances()) {
@@ -174,7 +175,8 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         }
     }
 
-    private InstancesResult getInstanceResultSubset(InstancesResult resultSet, String filterBy, String orderBy,
+    private InstancesResult getInstanceResultSubset(InstancesResult resultSet, String filterBy,
+                                                    String orderBy, String sortOrder,
                                                     Integer offset, Integer numResults) {
 
         ArrayList<Instance> instanceSet = new ArrayList<Instance>();
@@ -193,7 +195,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
             return new InstancesResult(resultSet.getMessage(), new Instance[0]);
         }
         // Sort the ArrayList using orderBy
-        instanceSet = sortInstances(instanceSet, orderBy);
+        instanceSet = sortInstances(instanceSet, orderBy, sortOrder);
         return new InstancesResult(resultSet.getMessage(),
                 instanceSet.subList(offset, (offset+pageCount)).toArray(new Instance[pageCount]));
     }
@@ -256,7 +258,9 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         return instanceSet;
     }
 
-    private ArrayList<Instance> sortInstances(ArrayList<Instance> instanceSet, String orderBy) {
+    private ArrayList<Instance> sortInstances(ArrayList<Instance> instanceSet,
+                                              String orderBy, String sortOrder) {
+        final String order = getValidSortOrder(sortOrder, orderBy);
         if (orderBy.equals("status")) {
             Collections.sort(instanceSet, new Comparator<Instance>() {
                 @Override
@@ -267,14 +271,16 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
                     if (i2.getStatus() == null) {
                         i2.status = InstancesResult.WorkflowStatus.ERROR;
                     }
-                    return i1.getStatus().name().compareTo(i2.getStatus().name());
+                    return (order.equalsIgnoreCase("asc")) ? i1.getStatus().name().compareTo(i2.getStatus().name())
+                            : i2.getStatus().name().compareTo(i1.getStatus().name());
                 }
             });
         } else if (orderBy.equals("cluster")) {
             Collections.sort(instanceSet, new Comparator<Instance>() {
                 @Override
                 public int compare(Instance i1, Instance i2) {
-                    return i1.getCluster().compareTo(i2.getCluster());
+                    return (order.equalsIgnoreCase("asc")) ? i1.getCluster().compareTo(i2.getCluster())
+                            : i2.getCluster().compareTo(i1.getCluster());
                 }
             });
         } else if (orderBy.equals("startTime")){
@@ -283,7 +289,8 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
                 public int compare(Instance i1, Instance i2) {
                     Date start1 = (i1.getStartTime() == null) ? new Date(0) : i1.getStartTime();
                     Date start2 = (i2.getStartTime() == null) ? new Date(0) : i2.getStartTime();
-                    return start2.compareTo(start1); //default desc
+                    return (order.equalsIgnoreCase("asc")) ? start1.compareTo(start2)
+                            : start2.compareTo(start1);
                 }
             });
         } else if (orderBy.equals("endTime")) {
@@ -292,10 +299,12 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
                 public int compare(Instance i1, Instance i2) {
                     Date end1 = (i1.getEndTime() == null) ? new Date(0) : i1.getEndTime();
                     Date end2 = (i2.getEndTime() == null) ? new Date(0) : i2.getEndTime();
-                    return end2.compareTo(end1); //default desc
+                    return (order.equalsIgnoreCase("asc")) ? end1.compareTo(end2)
+                            : end2.compareTo(end1);
                 }
             });
         }//Default : no sort
+
         return instanceSet;
     }
 
@@ -328,7 +337,6 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
                                         String type, String entity, String startStr,
                                         String endStr, String colo,
                                         List<LifeCycle> lifeCycles) {
-
         checkColo(colo);
         checkType(type);
         try {
@@ -352,7 +360,6 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
                                            String type, String entity, String startStr,
                                            String endStr, String colo,
                                            List<LifeCycle> lifeCycles) {
-
         checkColo(colo);
         checkType(type);
         try {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
index 6fba6df..b3151b9 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
@@ -176,7 +176,7 @@ public abstract class AbstractSchedulableEntityManager extends AbstractInstanceM
      */
     public EntitySummaryResult getEntitySummary(String type, String cluster, String startDate, String endDate,
                                                 String fields, String filterBy, String filterTags,
-                                                String orderBy, Integer offset,
+                                                String orderBy, String sortOrder, Integer offset,
                                                 Integer resultsPerPage, Integer numInstances) {
         HashSet<String> fieldSet = new HashSet<String>(Arrays.asList(fields.toLowerCase().split(",")));
         Pair<Date, Date> startAndEndDates = getStartEndDatesForSummary(startDate, endDate);
@@ -187,7 +187,7 @@ public abstract class AbstractSchedulableEntityManager extends AbstractInstanceM
             entities = getEntities(type,
                     SchemaHelper.getDateFormat().format(startAndEndDates.first),
                     SchemaHelper.getDateFormat().format(startAndEndDates.second),
-                    cluster, filterBy, filterTags, orderBy, offset, resultsPerPage);
+                    cluster, filterBy, filterTags, orderBy, sortOrder, offset, resultsPerPage);
             colo = ((Cluster) configStore.get(EntityType.CLUSTER, cluster)).getColo();
         } catch (Exception e) {
             LOG.error("Failed to get entities", e);
@@ -199,7 +199,7 @@ public abstract class AbstractSchedulableEntityManager extends AbstractInstanceM
             InstancesResult instancesResult = getInstances(entity.getEntityType().name(), entity.getName(),
                     SchemaHelper.getDateFormat().format(startAndEndDates.first),
                     SchemaHelper.getDateFormat().format(startAndEndDates.second),
-                    colo, null, "", "", 0, numInstances);
+                    colo, null, "", "", "", 0, numInstances);
 
             /* ToDo - Use oozie bulk API after FALCON-591 is implemented
              *       getBulkInstances(entity, cluster,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
index bfce5f9..d172c3e 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
@@ -84,6 +84,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") final List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") final String sortOrder,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
@@ -91,7 +92,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).
                         invoke("getRunningInstances", type, entity, colo, lifeCycles,
-                                filterBy, orderBy, offset, resultsPerPage);
+                                filterBy, orderBy, sortOrder, offset, resultsPerPage);
             }
         }.execute(colo, type, entity);
     }
@@ -114,6 +115,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") final List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") final String sortOrder,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
@@ -121,7 +123,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("getInstances",
                         type, entity, startStr, endStr, colo, lifeCycles,
-                        filterBy, orderBy, offset, resultsPerPage);
+                        filterBy, orderBy, sortOrder, offset, resultsPerPage);
             }
         }.execute(colo, type, entity);
     }
@@ -140,6 +142,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") final List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") final String sortOrder,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
@@ -147,7 +150,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("getStatus",
                         type, entity, startStr, endStr, colo, lifeCycles,
-                        filterBy, orderBy, offset, resultsPerPage);
+                        filterBy, orderBy, sortOrder, offset, resultsPerPage);
             }
         }.execute(colo, type, entity);
     }
@@ -209,6 +212,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") final List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") final String filterBy,
             @DefaultValue("") @QueryParam("orderBy") final String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") final String sortOrder,
             @DefaultValue("0") @QueryParam("offset") final Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") final Integer resultsPerPage) {
         return new InstanceProxy() {
@@ -216,7 +220,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("getLogs",
                         type, entity, startStr, endStr, colo, runId, lifeCycles,
-                        filterBy, orderBy, offset, resultsPerPage);
+                        filterBy, orderBy, sortOrder, offset, resultsPerPage);
             }
         }.execute(colo, type, entity);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
index e69e531..fbccd6b 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
@@ -293,21 +293,6 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
     }
 
     @GET
-    @Path("list/{type}")
-    @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
-    @Override
-    public EntityList getEntityList(@PathParam("type") String type,
-                                    @DefaultValue("") @QueryParam("fields") String fields,
-                                    @DefaultValue("") @QueryParam("filterBy") String filterBy,
-                                    @DefaultValue("") @QueryParam("tags") String tags,
-                                    @DefaultValue("") @QueryParam("orderBy") String orderBy,
-                                    @DefaultValue("0") @QueryParam("offset") Integer offset,
-                                    @DefaultValue(DEFAULT_NUM_RESULTS)
-                                    @QueryParam("numResults") Integer resultsPerPage) {
-        return super.getEntityList(type, fields, filterBy, tags, orderBy, offset, resultsPerPage);
-    }
-
-    @GET
     @Path("definition/{type}/{entity}")
     @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN, MediaType.APPLICATION_JSON})
     @Override
@@ -406,6 +391,22 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     @GET
+    @Path("list/{type}")
+    @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
+    @Override
+    public EntityList getEntityList(@PathParam("type") String type,
+                                    @DefaultValue("") @QueryParam("fields") String fields,
+                                    @DefaultValue("") @QueryParam("filterBy") String filterBy,
+                                    @DefaultValue("") @QueryParam("tags") String tags,
+                                    @DefaultValue("") @QueryParam("orderBy") String orderBy,
+                                    @DefaultValue("asc") @QueryParam("sortOrder") String sortOrder,
+                                    @DefaultValue("0") @QueryParam("offset") Integer offset,
+                                    @DefaultValue(DEFAULT_NUM_RESULTS)
+                                    @QueryParam("numResults") Integer resultsPerPage) {
+        return super.getEntityList(type, fields, filterBy, tags, orderBy, sortOrder, offset, resultsPerPage);
+    }
+
+    @GET
     @Path("summary/{type}/{cluster}")
     @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
     @Monitored(event = "summary")
@@ -419,11 +420,12 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
             @DefaultValue("") @QueryParam("filterBy") final String entityFilter,
             @DefaultValue("") @QueryParam("tags") final String entityTags,
             @DefaultValue("") @QueryParam("orderBy") final String entityOrderBy,
+            @DefaultValue("asc") @QueryParam("sortOrder") String entitySortOrder,
             @DefaultValue("0") @QueryParam("offset") final Integer entityOffset,
             @DefaultValue("10") @QueryParam("numResults") final Integer numEntities,
             @DefaultValue("7") @QueryParam("numInstances") final Integer numInstanceResults) {
-        return super.getEntitySummary(type, cluster, startStr, endStr, entityFields, entityFilter, entityTags,
-                entityOrderBy, entityOffset, numEntities, numInstanceResults);
+        return super.getEntitySummary(type, cluster, startStr, endStr, entityFields, entityFilter,
+                entityTags, entityOrderBy, entitySortOrder, entityOffset, numEntities, numInstanceResults);
     }
     //RESUME CHECKSTYLE CHECK ParameterNumberCheck
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
index 0b55eb3..f7b4f45 100644
--- a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
+++ b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
@@ -118,7 +118,7 @@ public class EntityManagerTest extends AbstractEntityManager {
         Entity process2 = buildProcess("processAuthUser", System.getProperty("user.name"), "", "");
         configStore.publish(EntityType.PROCESS, process2);
 
-        EntityList entityList = this.getEntityList("process", "", "", "", "", 0, 10);
+        EntityList entityList = this.getEntityList("process", "", "", "", "", "asc", 0, 10);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 2);
 
@@ -127,7 +127,7 @@ public class EntityManagerTest extends AbstractEntityManager {
          */
         StartupProperties.get().setProperty("falcon.security.authorization.enabled", "true");
         CurrentUser.authenticate(System.getProperty("user.name"));
-        entityList = this.getEntityList("process", "", "", "", "", 0, 10);
+        entityList = this.getEntityList("process", "", "", "", "", "desc", 0, 10);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 2);
 
@@ -135,7 +135,7 @@ public class EntityManagerTest extends AbstractEntityManager {
          * Only one entity should be returned when the auth is enabled.
          */
         CurrentUser.authenticate("fakeUser");
-        entityList = this.getEntityList("process", "", "", "", "", 0, 10);
+        entityList = this.getEntityList("process", "", "", "", "", "", 0, 10);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 1);
 
@@ -165,17 +165,17 @@ public class EntityManagerTest extends AbstractEntityManager {
         configStore.publish(EntityType.PROCESS, process4);
 
         EntityList entityList = this.getEntityList("process", "tags", "PIPELINES:dataReplicationPipeline",
-                "", "name", 1, 2);
+                "", "name", "desc", 1, 2);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 2);
-        Assert.assertEquals(entityList.getElements()[1].name, "process4");
-        Assert.assertEquals(entityList.getElements()[1].tag.size(), 1);
-        Assert.assertEquals(entityList.getElements()[1].tag.get(0), "owner=producer@xyz.com");
+        Assert.assertEquals(entityList.getElements()[1].name, "process1");
+        Assert.assertEquals(entityList.getElements()[1].tag.size(), 2);
+        Assert.assertEquals(entityList.getElements()[1].tag.get(0), "consumer=consumer@xyz.com");
         Assert.assertEquals(entityList.getElements()[0].status, null);
 
 
         entityList = this.getEntityList("process", "pipelines", "",
-                "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", 0, 2);
+                "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", "", 0, 2);
         Assert.assertNotNull(entityList.getElements());
         Assert.assertEquals(entityList.getElements().length, 2);
         Assert.assertEquals(entityList.getElements()[1].name, "process2");
@@ -184,13 +184,22 @@ public class EntityManagerTest extends AbstractEntityManager {
         Assert.assertEquals(entityList.getElements()[0].tag, null);
 
         entityList = this.getEntityList("process", "pipelines", "",
-                "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", 10, 2);
+                "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", "", 10, 2);
         Assert.assertEquals(entityList.getElements().length, 0);
 
         // Test negative value for numResults, should throw an exception.
         try {
             this.getEntityList("process", "pipelines", "",
-                    "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", 10, -1);
+                    "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", "", 10, -1);
+            Assert.assertTrue(false);
+        } catch (Throwable e) {
+            Assert.assertTrue(true);
+        }
+
+        // Test invalid entry for sortOrder
+        try {
+            this.getEntityList("process", "pipelines", "",
+                    "consumer=consumer@xyz.com, owner=producer@xyz.com", "name", "invalid", 10, 2);
             Assert.assertTrue(false);
         } catch (Throwable e) {
             Assert.assertTrue(true);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
index a2aaac2..7a7d993 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
@@ -47,9 +47,11 @@ public class InstanceManager extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") String sortOrder,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
-        return super.getRunningInstances(type, entity, colo, lifeCycles, filterBy, orderBy, offset, resultsPerPage);
+        return super.getRunningInstances(type, entity, colo, lifeCycles, filterBy,
+                orderBy, sortOrder, offset, resultsPerPage);
     }
 
     /*
@@ -70,10 +72,11 @@ public class InstanceManager extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") String sortOrder,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getInstances(type, entity, startStr, endStr, colo, lifeCycles,
-                filterBy, orderBy, offset, resultsPerPage);
+                filterBy, orderBy, sortOrder, offset, resultsPerPage);
     }
 
     @GET
@@ -90,10 +93,11 @@ public class InstanceManager extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") String sortOrder,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getStatus(type, entity, startStr, endStr, colo, lifeCycles,
-                filterBy, orderBy, offset, resultsPerPage);
+                filterBy, orderBy, sortOrder, offset, resultsPerPage);
     }
 
     @GET
@@ -125,10 +129,11 @@ public class InstanceManager extends AbstractInstanceManager {
             @Dimension("lifecycle") @QueryParam("lifecycle") List<LifeCycle> lifeCycles,
             @DefaultValue("") @QueryParam("filterBy") String filterBy,
             @DefaultValue("") @QueryParam("orderBy") String orderBy,
+            @DefaultValue("") @QueryParam("sortOrder") String sortOrder,
             @DefaultValue("0") @QueryParam("offset") Integer offset,
             @DefaultValue(DEFAULT_NUM_RESULTS) @QueryParam("numResults") Integer resultsPerPage) {
         return super.getLogs(type, entity, startStr, endStr, colo, runId, lifeCycles,
-                filterBy, orderBy, offset, resultsPerPage);
+                filterBy, orderBy, sortOrder, offset, resultsPerPage);
     }
 
     @GET

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
index c10301b..c318bfb 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
@@ -53,6 +53,7 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
         return super.getDependencies(type, entity);
     }
 
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     @GET
     @Path("list/{type}")
     @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
@@ -63,13 +64,13 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
                                     @DefaultValue("") @QueryParam("filterBy") String filterBy,
                                     @DefaultValue("") @QueryParam("tags") String tags,
                                     @DefaultValue("") @QueryParam("orderBy") String orderBy,
+                                    @DefaultValue("asc") @QueryParam("sortOrder") String sortOrder,
                                     @DefaultValue("0") @QueryParam("offset") Integer offset,
                                     @DefaultValue(DEFAULT_NUM_RESULTS)
                                     @QueryParam("numResults") Integer resultsPerPage) {
-        return super.getEntityList(type, fields, filterBy, tags, orderBy, offset, resultsPerPage);
+        return super.getEntityList(type, fields, filterBy, tags, orderBy, sortOrder, offset, resultsPerPage);
     }
 
-    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     @GET
     @Path("summary/{type}/{cluster}")
     @Produces({MediaType.TEXT_XML, MediaType.APPLICATION_JSON})
@@ -84,11 +85,12 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
             @DefaultValue("") @QueryParam("filterBy") String entityFilter,
             @DefaultValue("") @QueryParam("tags")  String entityTags,
             @DefaultValue("") @QueryParam("orderBy") String entityOrderBy,
+            @DefaultValue("asc") @QueryParam("sortOrder") String entitySortOrder,
             @DefaultValue("0") @QueryParam("offset") Integer entityOffset,
             @DefaultValue("10") @QueryParam("numResults") Integer numEntities,
             @DefaultValue("7") @QueryParam("numInstances") Integer numInstanceResults) {
         return super.getEntitySummary(type, cluster, startStr, endStr, fields, entityFilter, entityTags,
-                entityOrderBy, entityOffset, numEntities, numInstanceResults);
+                entityOrderBy, entitySortOrder, entityOffset, numEntities, numInstanceResults);
     }
     //RESUME CHECKSTYLE CHECK ParameterNumberCheck
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
index 1612abb..82208b0 100644
--- a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
+++ b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
@@ -288,15 +288,24 @@ public class FalconCLIIT {
         Assert.assertEquals(0,
                 executeWithURL("entity -summary -type feed -cluster "+ overlay.get("cluster")
                         + " -fields status,tags -start " + START_INSTANCE
-                        + " -filterBy TYPE:FEED -orderBy name "
+                        + " -filterBy TYPE:FEED -orderBy name -sortOrder asc "
                         + " -offset 0 -numResults 1 -numInstances 5"));
         Assert.assertEquals(0,
                 executeWithURL("entity -summary -type process -fields status,pipelines"
                         + " -cluster " + overlay.get("cluster")
                         + " -start " + SchemaHelper.getDateFormat().format(new Date(0))
                         + " -end " + SchemaHelper.getDateFormat().format(new Date())
-                        + " -filterBy TYPE:PROCESS -orderBy name "
+                        + " -filterBy TYPE:PROCESS -orderBy name -sortOrder desc "
+                        + " -offset 0 -numResults 1 -numInstances 7"));
+
+        Assert.assertEquals(-1,
+                executeWithURL("entity -summary -type process -fields status,pipelines"
+                        + " -cluster " + overlay.get("cluster")
+                        + " -start " + SchemaHelper.getDateFormat().format(new Date(0))
+                        + " -end " + SchemaHelper.getDateFormat().format(new Date())
+                        + " -filterBy TYPE:PROCESS -orderBy name -sortOrder invalid "
                         + " -offset 0 -numResults 1 -numInstances 7"));
+
         // No start or end date
         Assert.assertEquals(0,
                 executeWithURL("entity -summary -type process -fields status,pipelines"
@@ -413,12 +422,23 @@ public class FalconCLIIT {
                 executeWithURL("instance -running -type feed -lifecycle eviction -name "
                         + overlay.get("outputFeedName")
                         + " -start " + startTimeString
-                        + " -orderBy startTime -offset 0 -numResults 1"));
+                        + " -orderBy startTime -sortOrder asc -offset 0 -numResults 1"));
         Assert.assertEquals(-1,
                 executeWithURL("instance -running -type feed -lifecycle eviction -name "
                         + overlay.get("outputFeedName")
                         + " -start " + SchemaHelper.getDateFormat().format(new Date())
                         + " -orderBy INVALID -offset 0 -numResults 1"));
+        Assert.assertEquals(0,
+                executeWithURL("instance -running -type feed -lifecycle eviction -name "
+                        + overlay.get("outputFeedName")
+                        + " -start " + startTimeString
+                        + " -orderBy startTime -sortOrder desc -offset 0 -numResults 1"));
+        Assert.assertEquals(-1,
+                executeWithURL("instance -running -type feed -lifecycle eviction -name "
+                        + overlay.get("outputFeedName")
+                        + " -start " + startTimeString
+                        + " -orderBy startTime -sortOrder invalid -offset 0 -numResults 1"));
+
         Assert.assertEquals(-1,
                 executeWithURL("instance -running -type feed -lifecycle eviction -name "
                         + overlay.get("outputFeedName")
@@ -430,7 +450,7 @@ public class FalconCLIIT {
                 executeWithURL("instance -running -type feed -lifecycle eviction -name "
                         + overlay.get("outputFeedName")
                         + " -start " + SchemaHelper.getDateFormat().format(new Date(10000))
-                        + " -orderBy startTime -offset 0 -numResults 1"));
+                        + " -orderBy startTime -sortOrder asc -offset 0 -numResults 1"));
         // testcase : end str is in future
         long futureTimeinMilliSecs = (new Date()).getTime()+ 86400000;
         Assert.assertEquals(0,
@@ -450,7 +470,7 @@ public class FalconCLIIT {
                         + overlay.get("processName")
                         + " -start "+ START_INSTANCE
                         + " -filterBy STATUS:SUCCEEDED,STARTEDAFTER:"+START_INSTANCE
-                        + " -orderBy startTime -offset 0 -numResults 1"));
+                        + " -orderBy startTime -sortOrder desc -offset 0 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("instance -list -type feed -lifecycle eviction -name "
                         + overlay.get("outputFeedName")
@@ -647,14 +667,15 @@ public class FalconCLIIT {
                         + " -offset 0 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("entity -list -type process -fields status "
-                        + " -filterBy STATUS:SUBMITTED,TYPE:process -orderBy name -offset 1 -numResults 1"));
+                        + " -filterBy STATUS:SUBMITTED,TYPE:process -orderBy name "
+                        + " -sortOrder asc -offset 1 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("entity -list -type process -fields status,pipelines "
                         + " -filterBy STATUS:SUBMITTED,type:process -orderBy name -offset 1 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("entity -list -type process -fields status,pipelines "
                         + " -filterBy STATUS:SUBMITTED,pipelines:testPipeline "
-                        + " -orderBy name -offset 1 -numResults 1"));
+                        + " -orderBy name -sortOrder desc -offset 1 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("entity -list -type process -fields status,tags "
                         + " -tags owner=producer@xyz.com,department=forecasting "
@@ -670,7 +691,11 @@ public class FalconCLIIT {
                         + " -filterBy INVALID:FILTER,TYPE:process -orderBy name -offset 1 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("entity -definition -type cluster -name " + overlay.get("cluster")));
-
+        Assert.assertEquals(-1,
+                executeWithURL("entity -list -type process -fields status,tags "
+                        + " -tags owner=producer@xyz.com,department=forecasting "
+                        + " -filterBy STATUS:SUBMITTED,type:process "
+                        + " -orderBy name -sortOrder invalid -offset 1 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("instance -status -type feed -name "
                         + overlay.get("outputFeedName") + " -start " + START_INSTANCE));
@@ -808,12 +833,31 @@ public class FalconCLIIT {
                 executeWithURL("instance -logs -type process -name "
                         + overlay.get("processName")
                         + " -start " + START_INSTANCE + " -end " + START_INSTANCE
-                        + " -filterBy STATUS:SUCCEEDED -orderBy startTime -offset 0 -numResults 1"));
+                        + " -filterBy STATUS:SUCCEEDED -orderBy endtime "
+                        + " -sortOrder asc -offset 0 -numResults 1"));
+        Assert.assertEquals(0,
+                executeWithURL("instance -logs -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE
+                        + " -filterBy STATUS:SUCCEEDED -orderBy starttime "
+                        + " -sortOrder asc -offset 0 -numResults 1"));
+        Assert.assertEquals(0,
+                executeWithURL("instance -logs -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE
+                        + " -filterBy STATUS:SUCCEEDED -orderBy cluster "
+                        + " -sortOrder asc -offset 0 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("instance -logs -type process -name "
                         + overlay.get("processName")
                         + " -start " + START_INSTANCE + " -end " + START_INSTANCE
                         + " -filterBy STATUS:WAITING -orderBy startTime -offset 0 -numResults 1"));
+        Assert.assertEquals(-1,
+                executeWithURL("instance -logs -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE
+                        + " -filterBy STATUS:SUCCEEDED -orderBy endtime "
+                        + " -sortOrder invalid -offset 0 -numResults 1"));
         Assert.assertEquals(0,
                 executeWithURL("instance -logs -type process -name "
                         + overlay.get("processName")

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2ed0112e/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
index 4755c30..cfcc14f 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyIT.java
@@ -749,7 +749,7 @@ public class EntityManagerJerseyIT {
                 .path("api/entities/list/process/")
                 .queryParam("filterBy", "TYPE:PROCESS,STATUS:RUNNING")
                 .queryParam("tags", "owner=producer@xyz.com, department=forecasting")
-                .queryParam("orderBy", "name").queryParam("offset", "2")
+                .queryParam("orderBy", "name").queryParam("sortOrder", "desc").queryParam("offset", "2")
                 .queryParam("numResults", "2").queryParam("fields", "status,tags")
                 .header("Cookie", context.getAuthenticationToken())
                 .type(MediaType.TEXT_XML)
@@ -762,7 +762,8 @@ public class EntityManagerJerseyIT {
 
         response = context.service
                 .path("api/entities/list/process/")
-                .queryParam("orderBy", "name").queryParam("offset", "50").queryParam("numResults", "2")
+                .queryParam("orderBy", "name").queryParam("sortOrder", "asc")
+                .queryParam("offset", "50").queryParam("numResults", "2")
                 .header("Cookie", context.getAuthenticationToken())
                 .type(MediaType.TEXT_XML)
                 .accept(MediaType.TEXT_XML)


[35/41] FALCON-675 Request URLS moved from parameters into methods in falcon-regression

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
index 304549d..4768e68 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
@@ -27,7 +27,6 @@ import org.apache.falcon.regression.core.util.BundleUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -91,7 +90,7 @@ public class PrismProcessSnSTest extends BaseTestClass {
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //check if there is no criss cross
-        ServiceResponse response = prism.getProcessHelper().getStatus(URLS.STATUS_URL, process2);
+        ServiceResponse response = prism.getProcessHelper().getStatus(process2);
         logger.info(response.getMessage());
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
@@ -105,15 +104,13 @@ public class PrismProcessSnSTest extends BaseTestClass {
     public void testProcessSnSForSubmittedProcessOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitProcess(true);
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         bundles[1].submitProcess(true);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
@@ -129,8 +126,7 @@ public class PrismProcessSnSTest extends BaseTestClass {
     public void testProcessSnSForSubmittedProcessOnBothColosUsingColoHelper()
         throws Exception {
         bundles[0].submitProcess(true);
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         bundles[1].submitProcess(true);
@@ -138,8 +134,7 @@ public class PrismProcessSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
         bundles[1].submitProcess(true);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
         //check if there is no criss cross
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
@@ -162,8 +157,7 @@ public class PrismProcessSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
 
         //reschedule trial
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper()
-            .schedule(URLS.SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().schedule(process1));
         Assert.assertEquals(OozieUtil.getBundles(cluster2.getFeedHelper().getOozieClient(),
             Util.readEntityName(process1), EntityType.PROCESS).size(), 1);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
@@ -179,19 +173,17 @@ public class PrismProcessSnSTest extends BaseTestClass {
         //schedule both bundles
         bundles[0].submitAndScheduleProcess();
         bundles[1].submitAndScheduleProcess();
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().suspend(URLS.SUSPEND_URL, process1));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().suspend(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process1));
         Assert.assertEquals(OozieUtil.getBundles(cluster2OC, Util.readEntityName(process1),
             EntityType.PROCESS).size(), 1);
-        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(URLS.SUSPEND_URL, process1));
+        AssertUtil.assertSucceeded(cluster2.getProcessHelper().resume(process1));
 
-        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(URLS.SUSPEND_URL, process2));
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2));
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(process2));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process2));
 
         Assert.assertEquals(OozieUtil.getBundles(cluster1OC, Util.readEntityName(process2),
             EntityType.PROCESS).size(), 1);
@@ -212,26 +204,24 @@ public class PrismProcessSnSTest extends BaseTestClass {
 
         bundles[0].submitAndScheduleProcess();
         Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
-            .getStatus(URLS.STATUS_URL, process1)).getMessage(), cluster1Running);
+            .getStatus(process1)).getMessage(), cluster1Running);
         bundles[1].submitAndScheduleProcess();
         Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
-            .getStatus(URLS.STATUS_URL, process2)).getMessage(), cluster2Running);
+            .getStatus(process2)).getMessage(), cluster2Running);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(process1));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL, process2));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(process2));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1));
-        AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process1));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process2));
 
         Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
-            .getStatus(URLS.STATUS_URL, process1)).getMessage(),cluster1Running);
+            .getStatus(process1)).getMessage(),cluster1Running);
         Assert.assertEquals(Util.parseResponse(prism.getProcessHelper()
-            .getStatus(URLS.STATUS_URL, process2)).getMessage(), cluster2Running);
+            .getStatus(process2)).getMessage(), cluster2Running);
     }
 
     /**
@@ -240,8 +230,13 @@ public class PrismProcessSnSTest extends BaseTestClass {
     @Test(groups = {"prism", "0.2", "distributed"})
     public void testScheduleNonExistentProcessOnBothColos() throws Exception {
         Assert.assertEquals(Util.parseResponse(cluster2.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process1)).getStatusCode(), 404);
+            .submitAndSchedule(process1)).getStatusCode(), 404);
         Assert.assertEquals(Util.parseResponse(cluster1.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process2)).getStatusCode(), 404);
+            .submitAndSchedule(process2)).getStatusCode(), 404);
+    }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
index a14cd35..23e0bd3 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
@@ -25,7 +25,6 @@ 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.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -87,21 +86,19 @@ public class PrismProcessSuspendTest extends BaseTestClass {
 
 
         //suspend using prismHelper
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         Util.shutDownService(cluster1.getProcessHelper());
 
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
 
         for (int i = 0; i < 2; i++) {
             //suspend on the other one
             AssertUtil.assertSucceeded(prism.getProcessHelper()
-                .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                .suspend(bundles[1].getProcessData()));
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
             AssertUtil
@@ -117,23 +114,21 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcessUsingColoHelper(cluster2);
 
         //suspend using prismHelper
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //suspend on the other one
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         Assert.assertTrue(Util.parseResponse(prism.getProcessHelper()
-            .getStatus(URLS.STATUS_URL, bundles[0].getProcessData())).getMessage()
+            .getStatus(bundles[0].getProcessData())).getMessage()
             .contains("SUSPENDED"));
         Assert.assertTrue(Util.parseResponse(prism.getProcessHelper()
-            .getStatus(URLS.STATUS_URL, bundles[1].getProcessData())).getMessage()
+            .getStatus(bundles[1].getProcessData())).getMessage()
             .contains("RUNNING"));
     }
 
@@ -144,22 +139,18 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcessUsingColoHelper(cluster2);
 
         //delete using coloHelpers
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
 
         //suspend using prismHelper
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(Util.URLS.DELETE_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[1].getProcessData()));
         //suspend on the other one
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
     }
@@ -174,7 +165,7 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //suspend using prismHelper
             AssertUtil.assertSucceeded(prism.getProcessHelper()
-                .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+                .suspend(bundles[0].getProcessData()));
             //verify
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
@@ -185,7 +176,7 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //suspend on the other one
             AssertUtil.assertSucceeded(prism.getProcessHelper()
-                .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                .suspend(bundles[1].getProcessData()));
             AssertUtil
                 .checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.SUSPENDED);
             AssertUtil
@@ -195,15 +186,11 @@ public class PrismProcessSuspendTest extends BaseTestClass {
 
     @Test(groups = "embedded")
     public void testSuspendNonExistentProcessOnBothColos() throws Exception {
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
-
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
+
+        AssertUtil.assertFailed(cluster1.getProcessHelper().suspend(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().suspend(bundles[0].getProcessData()));
     }
 
     @Test(groups = "embedded")
@@ -211,15 +198,11 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         bundles[0].submitProcess(true);
         bundles[1].submitProcess(true);
 
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
 
-        AssertUtil.assertFailed(cluster1.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(cluster1.getProcessHelper().suspend(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().suspend(bundles[1].getProcessData()));
     }
 
     @Test(groups = {"prism", "0.2", "distributed"})
@@ -232,14 +215,12 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         Util.shutDownService(cluster1.getProcessHelper());
 
         //suspend using prismHelper
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         //suspend on the other one
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.SUSPENDED);
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.RUNNING);
     }
@@ -251,23 +232,19 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         bundles[1].submitAndScheduleProcessUsingColoHelper(cluster2);
 
         //delete using coloHelpers
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(Util.URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
         Util.shutDownService(cluster1.getProcessHelper());
 
         //suspend using prismHelper
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(Util.URLS.DELETE_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[1].getProcessData()));
         //suspend on the other one
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster1OC, EntityType.PROCESS, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.KILLED);
     }
@@ -278,13 +255,10 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         restartRequired = true;
         Util.shutDownService(cluster1.getProcessHelper());
 
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
 
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().suspend(bundles[0].getProcessData()));
     }
 
     @Test(groups = "distributed")
@@ -295,12 +269,9 @@ public class PrismProcessSuspendTest extends BaseTestClass {
 
         Util.shutDownService(cluster1.getProcessHelper());
 
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getProcessData()));
-        AssertUtil.assertFailed(
-            prism.getProcessHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
-        AssertUtil.assertFailed(cluster2.getProcessHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[0].getProcessData()));
+        AssertUtil.assertFailed(prism.getProcessHelper().suspend(bundles[1].getProcessData()));
+        AssertUtil.assertFailed(cluster2.getProcessHelper().suspend(bundles[1].getProcessData()));
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
index cdd02da..b11d586 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
@@ -26,7 +26,6 @@ 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.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
@@ -84,7 +83,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
         List<String> beforeSubmit = cluster1.getClusterHelper().getStoreInfo();
         try {
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+            prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         } catch (ConnectException e) {
             Assert.assertTrue(e.getMessage().contains("Connection to "
                 + prism.getClusterHelper().getHostname() + " refused"), e.getMessage());
@@ -97,8 +96,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
     @Test(groups = "distributed")
     public void submitCluster_resubmitDiffContent() throws Exception {
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
         List<String> beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
@@ -106,7 +104,7 @@ public class PrismSubmitTest extends BaseTestClass {
         bundles[0].setCLusterWorkingPath(bundles[0].getClusters().get(0), randomHDFSPath);
         logger.info("modified cluster Data: "
             + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
@@ -121,15 +119,14 @@ public class PrismSubmitTest extends BaseTestClass {
         Util.shutDownService(cluster1.getClusterHelper());
         TimeUtil.sleepSeconds(30);
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
 
         Util.startService(cluster1.getClusterHelper());
         TimeUtil.sleepSeconds(30);
 
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
     }
 
@@ -137,14 +134,13 @@ public class PrismSubmitTest extends BaseTestClass {
     public void submitProcess_1ColoDownAfter2FeedSubmitStartAfterProcessSubmitAnsDeleteProcess()
         throws Exception {
         restartRequired = true;
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(0));
+        r = prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(1));
+        r = prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(1));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         Util.shutDownService(cluster1.getClusterHelper());
@@ -155,11 +151,9 @@ public class PrismSubmitTest extends BaseTestClass {
         List<String> beforeSubmitCluster2 = cluster2.getProcessHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getProcessHelper().getStoreInfo();
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(bundles[0].getProcessData()));
 
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(bundles[0].getProcessData()));
         AssertUtil.assertFailed(r);
         List<String> afterSubmitCluster1 = cluster1.getProcessHelper().getStoreInfo();
         List<String> afterSubmitCluster2 = cluster2.getProcessHelper().getStoreInfo();
@@ -177,7 +171,7 @@ public class PrismSubmitTest extends BaseTestClass {
         beforeSubmitCluster2 = cluster2.getProcessHelper().getStoreInfo();
         beforeSubmitPrism = prism.getProcessHelper().getStoreInfo();
 
-        r = prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        r = prism.getProcessHelper().delete(bundles[0].getProcessData());
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         afterSubmitCluster1 = cluster1.getProcessHelper().getStoreInfo();
@@ -193,18 +187,17 @@ public class PrismSubmitTest extends BaseTestClass {
     @Test(groups = "distributed")
     public void submitProcess_ideal() throws Exception {
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> beforeSubmitCluster1 = cluster1.getFeedHelper().getStoreInfo();
         List<String> beforeSubmitCluster2 = cluster2.getFeedHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getFeedHelper().getStoreInfo();
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(0));
+        r = prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(1));
+        r = prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(1));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> afterSubmitCluster1 = cluster1.getFeedHelper().getStoreInfo();
@@ -219,7 +212,7 @@ public class PrismSubmitTest extends BaseTestClass {
         beforeSubmitCluster2 = cluster2.getProcessHelper().getStoreInfo();
         beforeSubmitPrism = prism.getProcessHelper().getStoreInfo();
 
-        r = prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+        r = prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         afterSubmitCluster1 = cluster1.getProcessHelper().getStoreInfo();
@@ -242,8 +235,7 @@ public class PrismSubmitTest extends BaseTestClass {
         List<String> beforeSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getClusterHelper().getStoreInfo();
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
 
         List<String> afterSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
@@ -264,7 +256,7 @@ public class PrismSubmitTest extends BaseTestClass {
         beforeSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
         beforeSubmitPrism = prism.getClusterHelper().getStoreInfo();
 
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         afterSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
         afterSubmitPrism = prism.getClusterHelper().getStoreInfo();
@@ -277,15 +269,14 @@ public class PrismSubmitTest extends BaseTestClass {
 
     @Test(groups = "distributed")
     public void submitCluster_1prism1coloSubmitDeleted() throws Exception {
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
-        prism.getClusterHelper().delete(URLS.DELETE_URL, bundles[0].getClusters().get(0));
+        prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
+        prism.getClusterHelper().delete(bundles[0].getClusters().get(0));
 
         List<String> beforeSubmitCluster1 = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getClusterHelper().getStoreInfo();
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         List<String> afterSubmitCluster1 = cluster1.getClusterHelper().getStoreInfo();
         List<String> afterSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
@@ -303,8 +294,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
     @Test(groups = "embedded")
     public void submitProcess_woClusterSubmit() throws Exception {
-        ServiceResponse r =
-            prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+        ServiceResponse r = prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
 
         Assert.assertTrue(r.getMessage().contains("FAILED"));
         Assert.assertTrue(r.getMessage().contains("is not registered"));
@@ -312,11 +302,10 @@ public class PrismSubmitTest extends BaseTestClass {
 
     @Test(groups = "embedded")
     public void submitProcess_woFeedSubmit() throws Exception {
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
-        r = prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+        r = prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
         Assert.assertTrue(r.getMessage().contains("FAILED"));
         Assert.assertTrue(r.getMessage().contains("is not registered"));
     }
@@ -336,8 +325,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
         bundles[1].setCLusterColo(cluster2.getClusterHelper().getColoName());
         logger.info("cluster b2: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[1].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
 
         List<String> parCluster1 = cluster1.getClusterHelper().getStoreInfo();
@@ -354,7 +342,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
         bundles[0].setCLusterColo(cluster1.getClusterHelper().getColoName());
         logger.info("cluster b1: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> afterCluster1 = cluster1.getClusterHelper().getStoreInfo();
@@ -374,8 +362,7 @@ public class PrismSubmitTest extends BaseTestClass {
         Util.shutDownService(cluster1.getClusterHelper());
         List<String> beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         List<String> afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> afterSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
@@ -388,7 +375,7 @@ public class PrismSubmitTest extends BaseTestClass {
         TimeUtil.sleepSeconds(30);
         beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
         afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         afterSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
@@ -402,8 +389,7 @@ public class PrismSubmitTest extends BaseTestClass {
         restartRequired = true;
         Util.shutDownService(cluster1.getClusterHelper());
         TimeUtil.sleepSeconds(30);
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
         Util.startService(cluster1.getClusterHelper());
@@ -412,7 +398,7 @@ public class PrismSubmitTest extends BaseTestClass {
         List<String> beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
         bundles[0].setCLusterWorkingPath(bundles[0].getClusters().get(0), randomHDFSPath);
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
@@ -427,13 +413,12 @@ public class PrismSubmitTest extends BaseTestClass {
         restartRequired = true;
         Util.shutDownService(cluster1.getClusterHelper());
         TimeUtil.sleepSeconds(30);
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
 
         List<String> beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
-        r = prism.getClusterHelper().delete(URLS.DELETE_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().delete(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
 
         List<String> afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
@@ -442,15 +427,14 @@ public class PrismSubmitTest extends BaseTestClass {
         AssertUtil.compareDataStoreStates(beforeSubmitPrism, afterSubmitPrism,
             Util.readEntityName(bundles[0].getClusters().get(0)), -1);
 
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
     }
 
     @Test(groups = "distributed")
     public void submitCluster_submitPartialDeleted() throws Exception {
         restartRequired = true;
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
         TimeUtil.sleepSeconds(30);
@@ -460,7 +444,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
         List<String> beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
-        r = prism.getClusterHelper().delete(URLS.DELETE_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().delete(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("PARTIAL"));
         List<String> afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> afterSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
@@ -473,7 +457,7 @@ public class PrismSubmitTest extends BaseTestClass {
 
         beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
         afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         afterSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
@@ -484,13 +468,12 @@ public class PrismSubmitTest extends BaseTestClass {
 
     @Test(groups = "embedded")
     public void submitCluster_resubmitAlreadySucceeded() throws Exception {
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> beforeSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = cluster2.getClusterHelper().getStoreInfo();
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         List<String> afterSubmitCluster = cluster1.getClusterHelper().getStoreInfo();
@@ -505,8 +488,7 @@ public class PrismSubmitTest extends BaseTestClass {
         List<String> beforeSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getClusterHelper().getStoreInfo();
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         List<String> afterSubmitCluster1 = cluster1.getClusterHelper().getStoreInfo();
         List<String> afterSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
@@ -523,14 +505,13 @@ public class PrismSubmitTest extends BaseTestClass {
 
     @Test(groups = "embedded")
     public void submitCluster_1prism1coloAlreadySubmitted() throws Exception {
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         List<String> beforeSubmitCluster1 = cluster1.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getClusterHelper().getStoreInfo();
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
 
         List<String> afterSubmitCluster1 = cluster1.getClusterHelper().getStoreInfo();
         List<String> afterSubmitCluster2 = cluster2.getClusterHelper().getStoreInfo();
@@ -545,11 +526,10 @@ public class PrismSubmitTest extends BaseTestClass {
     @Test
     public void submitProcess_1ColoDownAfter1FeedSubmitStartAfter2feed() throws Exception {
         restartRequired = true;
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"), r.getMessage());
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(0));
+        r = prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"), r.getMessage());
 
         Util.shutDownService(cluster1.getClusterHelper());
@@ -559,7 +539,7 @@ public class PrismSubmitTest extends BaseTestClass {
         List<String> beforeSubmitCluster2 = cluster2.getFeedHelper().getStoreInfo();
         List<String> beforeSubmitPrism = prism.getFeedHelper().getStoreInfo();
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(1));
+        r = prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(1));
         Assert.assertTrue(r.getMessage().contains("FAILED"));
 
         List<String> afterSubmitCluster1 = cluster1.getFeedHelper().getStoreInfo();
@@ -578,7 +558,7 @@ public class PrismSubmitTest extends BaseTestClass {
         beforeSubmitCluster2 = cluster2.getProcessHelper().getStoreInfo();
         beforeSubmitPrism = prism.getProcessHelper().getStoreInfo();
 
-        r = prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getProcessData());
+        r = prism.getProcessHelper().submitEntity(bundles[0].getProcessData());
         Assert.assertTrue(r.getMessage().contains("FAILED"), r.getMessage());
 
         afterSubmitCluster1 = cluster1.getProcessHelper().getStoreInfo();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
index 139b472..dac4cf6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
@@ -27,7 +27,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -95,12 +94,10 @@ public class RescheduleKilledProcessTest extends BaseTestClass {
 
         bundles[0].submitFeedsScheduleProcess(prism);
 
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(URLS.SCHEDULE_URL,
-            bundles[0].getProcessData()));
+        String processData = bundles[0].getProcessData();
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(processData));
     }
 
     /**
@@ -118,19 +115,13 @@ public class RescheduleKilledProcessTest extends BaseTestClass {
         logger.info("process: " + Util.prettyPrintXml(bundles[0].getProcessData()));
 
         bundles[0].submitFeedsScheduleProcess(prism);
-
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(URLS.SCHEDULE_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(URLS.DELETE_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(URLS.SUBMIT_URL,
-            bundles[0].getProcessData()));
-        AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(URLS.SCHEDULE_URL,
-            bundles[0].getProcessData()));
+        String processData = bundles[0].getProcessData();
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().delete(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitEntity(processData));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(processData));
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
index 3bba852..d168ed4 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
@@ -29,7 +29,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
@@ -113,12 +112,12 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
     public void rescheduleSucceeded() throws Exception {
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
         checkNotFoundDefinition(bundles[0].getProcessData());
 
         //submit and schedule process again
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(bundles[0].getProcessData()));
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
     }
@@ -135,12 +134,12 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
     public void rescheduleFailed() throws Exception {
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
         checkNotFoundDefinition(bundles[0].getProcessData());
 
         //submit and schedule process again
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(bundles[0].getProcessData()));
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
     }
@@ -161,12 +160,12 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.DONEWITHERROR);
 
         //delete the process
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
         checkNotFoundDefinition(bundles[0].getProcessData());
 
         //submit and schedule process again
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(bundles[0].getProcessData()));
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
     }
@@ -177,14 +176,14 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
      **/
     @Test(enabled = true)
     public void rescheduleKilled() throws Exception {
-        prism.getProcessHelper().delete(URLS.DELETE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.KILLED);
         checkNotFoundDefinition(bundles[0].getProcessData());
 
         //submit and schedule process again
         AssertUtil.assertSucceeded(prism.getProcessHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getProcessData()));
+            .submitAndSchedule(bundles[0].getProcessData()));
         InstanceUtil
             .waitForBundleToReachState(cluster, bundles[0].getProcessName(), Status.SUCCEEDED);
     }
@@ -201,8 +200,7 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
      */
     private void checkNotFoundDefinition(String process)
         throws URISyntaxException, IOException, AuthenticationException, JAXBException {
-        ServiceResponse r = prism.getProcessHelper()
-            .getEntityDefinition(URLS.GET_ENTITY_DEFINITION, process);
+        ServiceResponse r = prism.getProcessHelper().getEntityDefinition(process);
         Assert.assertTrue(r.getMessage().contains("(process) not found"));
         AssertUtil.assertFailed(r);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
index 8bce3de..d113554 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
@@ -35,7 +35,6 @@ import org.apache.falcon.regression.core.util.MathUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
@@ -113,8 +112,7 @@ public class RetentionTest extends BaseTestClass {
         final FeedMerlin feedObject = new FeedMerlin(bundles[0].getInputFeedFromBundle());
         feedObject.setRetentionValue(retentionUnit.getValue() + "(" + retentionPeriod + ")");
 
-        final ServiceResponse response = prism.getFeedHelper()
-            .submitEntity(URLS.SUBMIT_URL, feedObject.toString());
+        final ServiceResponse response = prism.getFeedHelper().submitEntity(feedObject.toString());
         if (retentionPeriod > 0) {
             AssertUtil.assertSucceeded(response);
 
@@ -174,7 +172,7 @@ public class RetentionTest extends BaseTestClass {
         //get Data created in the cluster
         List<String> initialData = Util.getHadoopDataFromDir(clusterFS, feed, testHDFSDir);
 
-        cluster.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        cluster.getFeedHelper().schedule(feed);
         logger.info(cluster.getClusterHelper().getActiveMQ());
         final String feedName = Util.readEntityName(feed);
         logger.info(feedName);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
index cd7d7bd..378cbbd 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
@@ -177,8 +177,7 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
         //submit and schedule feed
-        ServiceResponse r =
-            prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
         InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
@@ -232,10 +231,8 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             processBundle.submitBundle(prism);
 
             //schedule of 2 cluster
-            cluster1.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL,
-                processBundle.getProcessData());
-            cluster2.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL,
-                processBundle.getProcessData());
+            cluster1.getProcessHelper().schedule(processBundle.getProcessData());
+            cluster2.getProcessHelper().schedule(processBundle.getProcessData());
             InstanceUtil.waitTillInstancesAreCreated(cluster2, processBundle.getProcessData(), 0);
 
             //shut down cluster2
@@ -319,13 +316,13 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             logger.info("feed: " + Util.prettyPrintXml(feed));
 
             //submit feed on all 3 clusters
-            ServiceResponse r = prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, feed);
+            ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
             AssertUtil.assertSucceeded(r);
 
             //schedule feed of cluster1 and cluster2
-            r = cluster1.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed);
+            r = cluster1.getFeedHelper().schedule(feed);
             AssertUtil.assertSucceeded(r);
-            r = cluster2.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed);
+            r = cluster2.getFeedHelper().schedule(feed);
             AssertUtil.assertSucceeded(r);
             InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
 
@@ -441,10 +438,10 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             Util.readEntityName(processBundle.getClusters().get(0)), ClusterType.SOURCE,
             null, baseTestDir + "/replication" + MINUTE_DATE_PATTERN);
 
-        ServiceResponse r = prism.getClusterHelper().submitEntity(Util.URLS.SUBMIT_URL,
+        ServiceResponse r = prism.getClusterHelper().submitEntity(
             processBundle.getClusters().get(0));
         AssertUtil.assertSucceeded(r);
-        r = prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(r);
         InstanceUtil.waitTillInstancesAreCreated(cluster1, feed, 0);
 
@@ -585,12 +582,10 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
             XmlUtil.createRtention("days(1000000)", ActionType.DELETE),
             Util.readEntityName(b.getClusters().get(0)), ClusterType.SOURCE, "",
             "/someTestPath" + MINUTE_DATE_PATTERN);
-        ServiceResponse r = prism.getClusterHelper().submitEntity(Util.URLS
-                .SUBMIT_URL,
+        ServiceResponse r = prism.getClusterHelper().submitEntity(
             b.getClusters().get(0));
         AssertUtil.assertSucceeded(r);
-        r = prism.getFeedHelper().submitAndSchedule(Util.URLS
-            .SUBMIT_AND_SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(r);
         return feed;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
index 7a69e58..d7ac724 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
@@ -111,7 +111,7 @@ public class LineageGraphTest extends BaseUITestClass {
         inputFeedName = bundles[0].getInputFeedNameFromBundle();
         outputFeedName = bundles[0].getOutputFeedNameFromBundle();
         /**schedule process, wait for instances to succeed*/
-        prism.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().schedule(bundles[0].getProcessData());
         InstanceUtil.waitTillInstanceReachState(clusterOC, bundles[0].getProcessName(), 3,
             CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
         /**get process instances*/

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
index 110e2e9..76173b7 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
@@ -179,7 +179,7 @@ public class ProcessUITest extends BaseUITestClass {
         String processName = Util.readEntityName(process);
         softAssert.assertEquals(page.getEntityStatus(processName),
                 EntitiesPage.EntityStatus.SUBMITTED, "Process status should be SUBMITTED");
-        prism.getProcessHelper().schedule(Util.URLS.SCHEDULE_URL, process);
+        prism.getProcessHelper().schedule(process);
 
         InstanceUtil.waitTillInstanceReachState(clusterOC, processName, 1,
             CoordinatorAction.Status.RUNNING, EntityType.PROCESS);


[29/41] git commit: FALCON-681 delete duplicate feed retention test from falcon regression contributed by SamarthG

Posted by ra...@apache.org.
FALCON-681 delete duplicate feed retention test from falcon regression contributed by SamarthG


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

Branch: refs/heads/FALCON-585
Commit: cb11ff2599894f7c97a8c8ffdef612aff45b4343
Parents: 9aad374
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Fri Sep 5 12:18:50 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Fri Sep 5 12:18:50 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   2 +-
 .../falcon/regression/core/util/BundleUtil.java |   4 -
 .../regression/prism/FeedRetentionTest.java     | 215 -------------------
 .../FETL-BillingRC.xml                          |  14 +-
 .../test/resources/impressionRC/cluster-0.1.xml |  43 ----
 .../impressionRC/feed-FETL-ImpressionRC.xml     |  45 ----
 .../impressionRC/feed-FETL-RequestRC.xml        |  46 ----
 .../resources/impressionRC/feed-FETL2-RRLog.xml |  33 ---
 .../process-FETL-ImpressionRC-Conversion.xml    |  45 ----
 9 files changed, 4 insertions(+), 443 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 0b4714a..20d5882 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -40,5 +40,5 @@ Trunk (Unreleased)
    mismatched (Raghav Kumar Gautam via Arpit Gupta)
 
   OPTIMIZATIONS
-
+  FALCON-681 delete duplicate feed retention test from falcon regression (SamarthG)
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
index 1f73523..a069dc5 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
@@ -72,10 +72,6 @@ public final class BundleUtil {
         return readBundleFromFolder("LocalDC_feedReplicaltion_BillingRC");
     }
 
-    public static Bundle readImpressionRCBundle() throws IOException {
-        return readBundleFromFolder("impressionRC");
-    }
-
     public static Bundle readUpdateBundle() throws IOException {
         return readBundleFromFolder("updateBundle");
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
deleted file mode 100644
index 666a62f..0000000
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.falcon.regression.prism;
-
-import org.apache.falcon.regression.core.bundle.Bundle;
-import org.apache.falcon.entity.v0.feed.ActionType;
-import org.apache.falcon.entity.v0.feed.ClusterType;
-import org.apache.falcon.regression.core.helpers.ColoHelper;
-import org.apache.falcon.regression.core.util.AssertUtil;
-import org.apache.falcon.regression.core.util.BundleUtil;
-import org.apache.falcon.regression.core.util.HadoopUtil;
-import org.apache.falcon.regression.core.util.InstanceUtil;
-import org.apache.falcon.regression.core.util.OSUtil;
-import org.apache.falcon.regression.core.util.TimeUtil;
-import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
-import org.apache.falcon.regression.core.util.XmlUtil;
-import org.apache.falcon.regression.testHelper.BaseTestClass;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.log4j.Logger;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-
-
-@Test(groups = "embedded")
-public class FeedRetentionTest extends BaseTestClass {
-
-    ColoHelper cluster1 = servers.get(0);
-    ColoHelper cluster2 = servers.get(1);
-    FileSystem cluster1FS = serverFS.get(0);
-    FileSystem cluster2FS = serverFS.get(1);
-    String impressionrcWorkflowDir = baseHDFSDir + "/FeedRetentionTest/impressionrc/";
-    String impressionrcWorkflowLibPath = impressionrcWorkflowDir + "lib";
-    private static final Logger logger = Logger.getLogger(FeedRetentionTest.class);
-
-    @BeforeClass(alwaysRun = true)
-    public void uploadWorkflow() throws Exception {
-        for (FileSystem fs : serverFS) {
-            fs.copyFromLocalFile(new Path(
-                OSUtil.getPath(OSUtil.RESOURCES, "workflows", "impression_rc_workflow.xml")),
-                new Path(impressionrcWorkflowDir + "workflow.xml"));
-            HadoopUtil.uploadDir(fs, impressionrcWorkflowLibPath, OSUtil.RESOURCES_OOZIE + "lib");
-        }
-    }
-
-    @BeforeMethod(alwaysRun = true)
-    public void setUp(Method method) throws Exception {
-        logger.info("test name: " + method.getName());
-        //getImpressionRC bundle
-        bundles[0] = BundleUtil.readImpressionRCBundle();
-        bundles[0].generateUniqueBundle();
-        bundles[0] = new Bundle(bundles[0], cluster1);
-        bundles[0].setProcessWorkflow(impressionrcWorkflowDir);
-
-        bundles[1] = BundleUtil.readImpressionRCBundle();
-        bundles[1].generateUniqueBundle();
-        bundles[1] = new Bundle(bundles[1], cluster2);
-        bundles[1].setProcessWorkflow(impressionrcWorkflowDir);
-    }
-
-    @AfterMethod(alwaysRun = true)
-    public void tearDown() {
-        removeBundles();
-    }
-
-    /**
-     * submit 2 clusters
-     * submit and schedule feed on above 2 clusters, both having different locations
-     * submit and schedule process having the above feed as output feed and running on 2
-     * clusters
-     */
-    @Test(enabled = true)
-    public void testRetentionClickRC_2Colo() throws Exception {
-        String inputPath = baseHDFSDir + "/testInput/";
-        String inputData = inputPath + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-        String outputPathTemplate = baseHDFSDir +
-            "/testOutput/op%d/ivoryRetention0%d/%s/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-        String cluster1Def = bundles[0].getClusters().get(0);
-        String cluster2Def = bundles[1].getClusters().get(0);
-        String cluster1Name = Util.readEntityName(cluster1Def);
-        String cluster2Name = Util.readEntityName(cluster2Def);
-
-        List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(
-            TimeUtil.getTimeWrtSystemTime(-5), TimeUtil.getTimeWrtSystemTime(10), 1);
-        HadoopUtil.flattenAndPutDataInFolder(cluster1FS, OSUtil.RESOURCES + "thriftRRMar0602.gz",
-            inputPath, dataDates);
-        HadoopUtil.flattenAndPutDataInFolder(cluster2FS, OSUtil.RESOURCES + "thriftRRMar0602.gz",
-            inputPath, dataDates);
-
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster1Def);
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, cluster2Def);
-
-        String feedOutput01 = bundles[0].getFeed("FETL-RequestRC");
-        feedOutput01 = InstanceUtil.setFeedCluster(feedOutput01,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-01-01T00:00Z"),
-            XmlUtil.createRtention("days(10000)", ActionType.DELETE), null,
-            ClusterType.SOURCE, null);
-
-        feedOutput01 = InstanceUtil.setFeedCluster(feedOutput01,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:10Z"),
-            XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            cluster1Name, ClusterType.SOURCE, "${cluster.colo}",
-            String.format(outputPathTemplate, 1, 1, "data"),
-            String.format(outputPathTemplate, 1, 1, "stats"),
-            String.format(outputPathTemplate, 1, 1, "meta"),
-            String.format(outputPathTemplate, 1, 1, "tmp"));
-
-        feedOutput01 = InstanceUtil.setFeedCluster(feedOutput01,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:25Z"),
-            XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            cluster2Name, ClusterType.SOURCE,"${cluster.colo}",
-            String.format(outputPathTemplate, 1, 2, "data"),
-            String.format(outputPathTemplate, 1, 2, "stats"),
-            String.format(outputPathTemplate, 1, 2, "meta"),
-            String.format(outputPathTemplate, 1, 2, "tmp"));
-
-        //submit the new output feed
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOutput01));
-
-        String feedOutput02 = bundles[0].getFeed("FETL-ImpressionRC");
-        feedOutput02 = InstanceUtil.setFeedCluster(feedOutput02,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-01-01T00:00Z"),
-            XmlUtil.createRtention("days(10000)", ActionType.DELETE), null,
-            ClusterType.SOURCE, null);
-
-        feedOutput02 = InstanceUtil.setFeedCluster(feedOutput02,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:10Z"),
-            XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            cluster1Name, ClusterType.SOURCE, "${cluster.colo}",
-            String.format(outputPathTemplate, 2, 1, "data"),
-            String.format(outputPathTemplate, 2, 1, "stats"),
-            String.format(outputPathTemplate, 2, 1, "meta"),
-            String.format(outputPathTemplate, 2, 1, "tmp"));
-
-        feedOutput02 = InstanceUtil.setFeedCluster(feedOutput02,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:25Z"),
-            XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            cluster2Name, ClusterType.SOURCE, "${cluster.colo}",
-            String.format(outputPathTemplate, 2, 2, "data"),
-            String.format(outputPathTemplate, 2, 2, "stats"),
-            String.format(outputPathTemplate, 2, 2, "meta"),
-            String.format(outputPathTemplate, 2, 2, "tmp"));
-
-        //submit the new output feed
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feedOutput02));
-
-        String feedInput = bundles[0].getFeed("FETL2-RRLog");
-        feedInput = InstanceUtil.setFeedCluster(feedInput,
-                XmlUtil.createValidity("2010-10-01T12:00Z", "2099-01-01T00:00Z"),
-                XmlUtil.createRtention("days(10000)", ActionType.DELETE), null,
-                ClusterType.SOURCE, null);
-
-        feedInput = InstanceUtil.setFeedCluster(feedInput,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:10Z"),
-            XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            cluster1Name, ClusterType.SOURCE, "${cluster.colo}", inputData);
-
-        feedInput = InstanceUtil.setFeedCluster(feedInput,
-            XmlUtil.createValidity("2010-10-01T12:00Z", "2099-10-01T12:25Z"),
-            XmlUtil.createRtention("minutes(5)", ActionType.DELETE),
-            cluster2Name, ClusterType.SOURCE, "${cluster.colo}", inputData);
-
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedInput));
-
-        String process = bundles[0].getProcessData();
-        process = InstanceUtil.setProcessCluster(process, null,
-            XmlUtil.createProcessValidity("2012-10-01T12:00Z", "2012-10-01T12:10Z"));
-
-        process = InstanceUtil.setProcessCluster(process, cluster1Name,
-            XmlUtil.createProcessValidity(TimeUtil.getTimeWrtSystemTime(-2),
-                TimeUtil.getTimeWrtSystemTime(5)));
-        process = InstanceUtil.setProcessCluster(process, cluster2Name,
-            XmlUtil.createProcessValidity(TimeUtil.getTimeWrtSystemTime(-2),
-                TimeUtil.getTimeWrtSystemTime(5)));
-
-        logger.info("process: " + Util.prettyPrintXml(process));
-        AssertUtil.assertSucceeded(
-            prism.getProcessHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, process));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOutput01));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOutput02));
-    }
-
-    @AfterClass(alwaysRun = true)
-    public void tearDownClass() throws IOException {
-        cleanTestDirs();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/resources/LocalDC_feedReplicaltion_BillingRC/FETL-BillingRC.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/resources/LocalDC_feedReplicaltion_BillingRC/FETL-BillingRC.xml b/falcon-regression/merlin/src/test/resources/LocalDC_feedReplicaltion_BillingRC/FETL-BillingRC.xml
index d7e0a91..eb31839 100755
--- a/falcon-regression/merlin/src/test/resources/LocalDC_feedReplicaltion_BillingRC/FETL-BillingRC.xml
+++ b/falcon-regression/merlin/src/test/resources/LocalDC_feedReplicaltion_BillingRC/FETL-BillingRC.xml
@@ -27,15 +27,7 @@
     <frequency>minutes(5)</frequency>
     <late-arrival cut-off="days(100000)"/>
     <clusters>
-        <cluster name="lhr1-emerald" type="target" partition="${cluster.colo}">
-            <validity start="2012-07-20T00:00Z" end="2099-07-16T00:00Z"/>
-            <retention limit="days(10000)" action="delete"/>
-        </cluster>
-        <cluster name="ua2-ruby" type="target" partition="${cluster.colo}">
-            <validity start="2012-07-16T00:00Z" end="2099-07-16T00:00Z"/>
-            <retention limit="days(10000)" action="delete"/>
-        </cluster>
-         <cluster name="prod-gloabl" type="source">
+         <cluster name="clusterName" type="source">
             <validity start="2012-07-20T03:00Z" end="2099-07-16T00:00Z"/>
             <retention limit="days(10000)" action="delete"/>
         </cluster>
@@ -45,8 +37,8 @@
         <location type="stats" path="/data/regression/fetlrc/billing/stats"/>
         <location type="meta" path="/data/regression/fetlrc/billing/metadata"/>
     </locations>
-    <ACL owner="fetl" group="group" permission="0x755"/>
-    <schema location="/databus/streams_local/click_rr/schema/" provider="protobuf"/>
+    <ACL owner="default" group="group" permission="0x755"/>
+    <schema location="/schemaLocaltion/" provider="protobuf"/>
 
     <properties>
     <property name="field1" value="value1" />

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/resources/impressionRC/cluster-0.1.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/resources/impressionRC/cluster-0.1.xml b/falcon-regression/merlin/src/test/resources/impressionRC/cluster-0.1.xml
deleted file mode 100755
index 144c3b4..0000000
--- a/falcon-regression/merlin/src/test/resources/impressionRC/cluster-0.1.xml
+++ /dev/null
@@ -1,43 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-
-<cluster colo="gs" description="" name="ua2-staging" xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="hftp://gs1001.grid.corp.inmobi.com:50070"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://gs1001.grid.corp.inmobi.com:54310"
-			version="0.20.2" />
-		<interface type="execute" endpoint="hdfs://gs1001.grid.corp.inmobi.com:54311" version="0.20.2" />
-		<interface type="workflow" endpoint="http://gs1001.grid.corp.inmobi.com:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://gs1001.grid.corp.inmobi.com:61616?daemon=true"
-			version="5.1.6" />
-		
-	</interfaces>
-	<locations>
-		<location name="staging" path="/projects/ivory/staging" />
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/projects/ivory/working" />
-	</locations>
-	<properties>
-		 <property name="colo.name" value="ua2"/>
-        <property name="hbase.zookeeper.quorum" value="10.14.117.33"/>
-        <property name="hbase.zookeeper.property.clientPort" value="2181"/>
-	</properties>
-</cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-ImpressionRC.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-ImpressionRC.xml b/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-ImpressionRC.xml
deleted file mode 100644
index 8b459ff..0000000
--- a/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-ImpressionRC.xml
+++ /dev/null
@@ -1,45 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  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.
-  -->
-
-<feed name="FETL-ImpressionRC" description="RC Impression file and hbase store values generated by iolib" xmlns="uri:falcon:feed:0.1">
-    <partitions>
-        <partition name="EventTime"/>
-        <partition name="PricingModel"/>
-    </partitions>
-    <groups>minutelyrcdata</groups>
-    <availabilityFlag>_SUCCESS</availabilityFlag>
-    <frequency>minutes(1)</frequency>
-    <clusters>
-        <cluster name="ua2-staging" >
-		<validity start="2010-12-27T06:00Z" end="2032-11-19T23:00Z"/>
-		<retention limit="days(5)" action="delete" />
-	</cluster>
-    </clusters>
-    <locations>
-        <location type="data" path="/data/fetl/impression/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-        <location type="stats" path="/data/fetl/stats/impression/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-        <location type="tmp" path="/data/fetl/staging/impression/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-    </locations>
-    <ACL owner="fetl" group="users" permission="0x755"/>
-    <schema location="/data/iolib/impressions/schema/" provider="protobuf"/>
-    <properties>
-	<property name="queueName" value="default" />
-	<property name="jobPriority" value="NORMAL" />
-    </properties>
-</feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-RequestRC.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-RequestRC.xml b/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-RequestRC.xml
deleted file mode 100644
index e9b47c6..0000000
--- a/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL-RequestRC.xml
+++ /dev/null
@@ -1,46 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  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.
-  -->
-
-<feed name="FETL-RequestRC" description="RC Request file generated by iolib" xmlns="uri:falcon:feed:0.1">
-    <partitions>	
-        <partition name="EventTime"/>
-        <partition name="Fill-Nofill"/>
-    </partitions>
-    <groups>minutelyrcdata</groups>
-    <availabilityFlag>_SUCCESS</availabilityFlag>
-    <frequency>minutes(1)</frequency>
-    <clusters>
-        <cluster name="ua2-staging" >
-		<validity start="2010-12-27T06:30Z" end="2099-12-27T07:00Z"/>
-	<retention limit="days(5)" action="delete" />
-	</cluster>
-    </clusters>
-    <locations>
-        <location type="data" path="/data/fetl/request/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-	<location type="stats" path="/data/fetl/stats/request/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-        <location type="tmp" path="/data/fetl/staging/request/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-    </locations>
-    <ACL owner="fetl" group="users" permission="0x755"/>
-    <schema location="/data/iolib/impressions/schema/" provider="protobuf"/>
-    <properties>
-	<property name="queueName" value="default" />
-	<property name="jobPriority" value="NORMAL" />
-    </properties>
-</feed>
-

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL2-RRLog.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL2-RRLog.xml b/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL2-RRLog.xml
deleted file mode 100644
index 2f986d8..0000000
--- a/falcon-regression/merlin/src/test/resources/impressionRC/feed-FETL2-RRLog.xml
+++ /dev/null
@@ -1,33 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  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.
-  -->
-
-<feed name="FETL2-RRLog" description="rr log generated by databus" xmlns="uri:falcon:feed:0.1">
-    <frequency>minutes(1)</frequency>
-    <clusters>
-		<cluster name="ua2-staging">
-			<validity start="2010-12-27T06:00Z" end="2032-10-01T12:00Z" />
-			<retention limit="days(5)" action="delete" />
-		</cluster>
-	</clusters>
-    <locations>
-        <location type="data" path="/databus/streams_local/rr/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
-    </locations>
-    <ACL owner="databus" group="users" permission="0x755"/>
-    <schema location="/databus/streams_local/rr/schema/" provider="protobuf"/>
-</feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/cb11ff25/falcon-regression/merlin/src/test/resources/impressionRC/process-FETL-ImpressionRC-Conversion.xml
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/resources/impressionRC/process-FETL-ImpressionRC-Conversion.xml b/falcon-regression/merlin/src/test/resources/impressionRC/process-FETL-ImpressionRC-Conversion.xml
deleted file mode 100644
index aff2f78..0000000
--- a/falcon-regression/merlin/src/test/resources/impressionRC/process-FETL-ImpressionRC-Conversion.xml
+++ /dev/null
@@ -1,45 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  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.
-  -->
-
-<process name="FETL-ImpressionRC-Conversion" xmlns="uri:falcon:process:0.1">
-    <clusters>
-        <cluster name="ua2-staging">
-            <validity start="2013-02-07T12:00Z" end="2019-02-07T12:10Z"/>
-        </cluster>
-    </clusters>
-    <parallel>1</parallel>
-    <order>FIFO</order>
-    <frequency>minutes(1)</frequency>
-    <inputs>
-        <input name="Input" feed="FETL2-RRLog" start="now(0,0)" end="now(0,0)"/>
-        <input name="Gate" feed="FETL2-RRLog" start="now(0,1)" end="now(0,1)"/>
-    </inputs>
-    <outputs>
-        <output name="Output" feed="FETL-RequestRC" instance="now(0,0)"/>
-	<output name="ImpressionPath" feed="FETL-ImpressionRC" instance="now(0,0)"/>
-    </outputs>
-    <properties>
-        <property name="logType" value="request"/>
-        <property name="jobPriority" value="VERY_HIGH" />
-	<property name="queueName" value="default" />
-    </properties>
-    <workflow path="/projects/localdc/lda/impressionrc-conversion"/>
-    <retry policy="exp-backoff" delay="minutes(1)" attempts="3"/>
-</process>
-


[11/41] git commit: FALCON-325 Process lineage information for Replication policies. Contributed by Sowmya Ramesh

Posted by ra...@apache.org.
FALCON-325 Process lineage information for Replication policies. Contributed by Sowmya Ramesh


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

Branch: refs/heads/FALCON-585
Commit: 23eed9f6e43c0b5b028e14130ab16afd5ac5179c
Parents: 305feb0
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Thu Aug 28 15:53:58 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Thu Aug 28 15:54:45 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../InstanceRelationshipGraphBuilder.java       |  44 +++-
 .../falcon/metadata/MetadataMappingService.java |   4 +-
 .../metadata/RelationshipGraphBuilder.java      |  13 +-
 .../falcon/metadata/RelationshipLabel.java      |   5 +-
 .../workflow/WorkflowExecutionContext.java      |  23 +-
 .../metadata/MetadataMappingServiceTest.java    | 258 ++++++++++++++-----
 .../feed/FeedReplicationCoordinatorBuilder.java |   4 +
 .../feed/OozieFeedWorkflowBuilderTest.java      |   6 +-
 .../falcon/oozie/process/AbstractTestBase.java  |  13 +-
 10 files changed, 290 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a358be4..075fe7e 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,9 @@ Trunk (Unreleased)
    FALCON-263 API to get workflow parameters. (pavan kumar kolamuri via Shwetha GS)
 
   IMPROVEMENTS
+   FALCON-325 Process lineage information for Replication policies
+   (Sowmya Ramesh via Venkatesh Seetharam)
+
    FALCON-474 Add Bulk APIs to drive the dashboard needs (Balu Vellanki via
    Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java b/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
index 735f87a..452872e 100644
--- a/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
+++ b/common/src/main/java/org/apache/falcon/metadata/InstanceRelationshipGraphBuilder.java
@@ -114,6 +114,12 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
 
     public void addInstanceToEntity(Vertex instanceVertex, String entityName,
                                     RelationshipType entityType, RelationshipLabel edgeLabel) {
+        addInstanceToEntity(instanceVertex, entityName, entityType, edgeLabel, null);
+    }
+
+    public void addInstanceToEntity(Vertex instanceVertex, String entityName,
+                                    RelationshipType entityType, RelationshipLabel edgeLabel,
+                                    String timestamp) {
         Vertex entityVertex = findVertex(entityName, entityType);
         LOG.info("Vertex exists? name={}, type={}, v={}", entityName, entityType, entityVertex);
         if (entityVertex == null) {
@@ -122,7 +128,7 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
             return;
         }
 
-        addEdge(instanceVertex, entityVertex, edgeLabel.getName());
+        addEdge(instanceVertex, entityVertex, edgeLabel.getName(), timestamp);
     }
 
     public void addOutputFeedInstances(WorkflowExecutionContext context,
@@ -166,6 +172,36 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
         }
     }
 
+    public void addReplicatedInstance(WorkflowExecutionContext context) throws FalconException {
+        String outputFeedNamesArg = context.getOutputFeedNames();
+        if ("NONE".equals(outputFeedNamesArg)) {
+            return; // there are no output feeds
+        }
+
+        String[] outputFeedNames = context.getOutputFeedNamesList();
+        String[] outputFeedInstancePaths = context.getOutputFeedInstancePathsList();
+        String targetClusterName = context.getClusterName();
+        String srcClusterName = context.getSrcClusterName();
+
+        // For replication there will be only one output feed name
+        String feedName = outputFeedNames[0];
+        String feedInstanceDataPath = outputFeedInstancePaths[0];
+
+        LOG.info("Computing feed instance for : name=" + feedName + ", path= "
+                + feedInstanceDataPath + ", in cluster: " + srcClusterName);
+        RelationshipType vertexType = RelationshipType.FEED_INSTANCE;
+        String feedInstanceName = getFeedInstanceName(feedName, srcClusterName, feedInstanceDataPath);
+        Vertex feedInstanceVertex = findVertex(feedInstanceName, vertexType);
+
+        LOG.info("Vertex exists? name={}, type={}, v={}", feedInstanceName, vertexType, feedInstanceVertex);
+        if (feedInstanceVertex == null) {
+            throw new IllegalStateException(vertexType + " instance vertex must exist " + feedInstanceName);
+        }
+
+        addInstanceToEntity(feedInstanceVertex, targetClusterName, RelationshipType.CLUSTER_ENTITY,
+                RelationshipLabel.FEED_CLUSTER_REPLICATED_EDGE, context.getTimeStampAsISO8601());
+    }
+
     private void addFeedInstance(Vertex processInstance, RelationshipLabel edgeLabel,
                                  WorkflowExecutionContext context, String feedName,
                                  String feedInstanceDataPath) throws FalconException {
@@ -193,7 +229,7 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
         }
     }
 
-    public String getFeedInstanceName(String feedName, String clusterName,
+    public static String getFeedInstanceName(String feedName, String clusterName,
                                       String feedInstancePath) throws FalconException {
         try {
             Feed feed = ConfigurationStore.get().get(EntityType.FEED, feedName);
@@ -209,14 +245,14 @@ public class InstanceRelationshipGraphBuilder extends RelationshipGraphBuilder {
         }
     }
 
-    private String getTableFeedInstanceName(Feed feed, String feedInstancePath,
+    private static String getTableFeedInstanceName(Feed feed, String feedInstancePath,
                                             Storage.TYPE storageType) throws URISyntaxException {
         CatalogStorage instanceStorage = (CatalogStorage) FeedHelper.createStorage(
                 storageType.name(), feedInstancePath);
         return feed.getName() + "/" + instanceStorage.toPartitionAsPath();
     }
 
-    private String getFileSystemFeedInstanceName(String feedInstancePath, Feed feed,
+    private static String getFileSystemFeedInstanceName(String feedInstancePath, Feed feed,
                                                  Cluster cluster) throws FalconException {
         Storage rawStorage = FeedHelper.createStorage(cluster, feed);
         String feedPathTemplate = rawStorage.getUriTemplate(LocationType.DATA);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
index a501e69..ab82ce1 100644
--- a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
+++ b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
@@ -288,9 +288,9 @@ public class MetadataMappingService
         instanceGraphBuilder.addInputFeedInstances(context, processInstance);
     }
 
-    private void onFeedInstanceReplicated(WorkflowExecutionContext context) {
+    private void onFeedInstanceReplicated(WorkflowExecutionContext context) throws FalconException {
         LOG.info("Adding replicated feed instance: {}", context.getNominalTimeAsISO8601());
-        // todo - tbd
+        instanceGraphBuilder.addReplicatedInstance(context);
     }
 
     private void onFeedInstanceEvicted(WorkflowExecutionContext context) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/common/src/main/java/org/apache/falcon/metadata/RelationshipGraphBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/RelationshipGraphBuilder.java b/common/src/main/java/org/apache/falcon/metadata/RelationshipGraphBuilder.java
index 898d914..d5685a5 100644
--- a/common/src/main/java/org/apache/falcon/metadata/RelationshipGraphBuilder.java
+++ b/common/src/main/java/org/apache/falcon/metadata/RelationshipGraphBuilder.java
@@ -109,8 +109,19 @@ public abstract class RelationshipGraphBuilder {
     }
 
     protected Edge addEdge(Vertex fromVertex, Vertex toVertex, String edgeLabel) {
+        return addEdge(fromVertex, toVertex, edgeLabel, null);
+    }
+
+    protected Edge addEdge(Vertex fromVertex, Vertex toVertex,
+                           String edgeLabel, String timestamp) {
         Edge edge = findEdge(fromVertex, toVertex, edgeLabel);
-        return edge != null ? edge : fromVertex.addEdge(edgeLabel, toVertex);
+
+        Edge edgeToVertex = edge != null ? edge : fromVertex.addEdge(edgeLabel, toVertex);
+        if (timestamp != null) {
+            edgeToVertex.setProperty(RelationshipProperty.TIMESTAMP.getName(), timestamp);
+        }
+
+        return edgeToVertex;
     }
 
     protected void removeEdge(Vertex fromVertex, Vertex toVertex, String edgeLabel) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java b/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
index 969640a..acd764f 100644
--- a/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
+++ b/common/src/main/java/org/apache/falcon/metadata/RelationshipLabel.java
@@ -36,7 +36,10 @@ public enum RelationshipLabel {
     CLUSTER_COLO("collocated"),
     USER("owned-by"),
     GROUPS("grouped-as"),
-    PIPELINES("part-of-pipeline");
+    PIPELINES("pipeline"),
+
+    // replication labels
+    FEED_CLUSTER_REPLICATED_EDGE("replicated-to");
 
     private final String name;
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
index f5bb782..c074484 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
@@ -54,7 +54,7 @@ public class WorkflowExecutionContext {
 
     public static final String OUTPUT_FEED_SEPARATOR = ",";
     public static final String INPUT_FEED_SEPARATOR = "#";
-
+    public static final String CLUSTER_NAME_SEPARATOR = ",";
 
     /**
      * Workflow execution status.
@@ -161,7 +161,26 @@ public class WorkflowExecutionContext {
     }
 
     public String getClusterName() {
-        return getValue(WorkflowExecutionArgs.CLUSTER_NAME);
+        String value =  getValue(WorkflowExecutionArgs.CLUSTER_NAME);
+        if (EntityOperations.REPLICATE != getOperation()) {
+            return value;
+        }
+
+        return value.split(CLUSTER_NAME_SEPARATOR)[0];
+    }
+
+    public String getSrcClusterName() {
+        String value =  getValue(WorkflowExecutionArgs.CLUSTER_NAME);
+        if (EntityOperations.REPLICATE != getOperation()) {
+            return value;
+        }
+
+        String[] parts = value.split(CLUSTER_NAME_SEPARATOR);
+        if (parts.length != 2) {
+            throw new IllegalArgumentException("Replicated cluster pair is missing in " + value);
+        }
+
+        return parts[1];
     }
 
     public String getEntityName() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
index 2b030fd..3f3f539 100644
--- a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
@@ -44,6 +44,7 @@ import org.apache.falcon.service.Services;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.falcon.workflow.WorkflowExecutionArgs;
 import org.apache.falcon.workflow.WorkflowExecutionContext;
+import static org.apache.falcon.workflow.WorkflowExecutionContext.EntityOperations;
 import org.apache.falcon.workflow.WorkflowJobEndNotificationService;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
@@ -66,12 +67,13 @@ public class MetadataMappingServiceTest {
     public static final String FALCON_USER = "falcon-user";
     private static final String LOGS_DIR = "target/log";
     private static final String NOMINAL_TIME = "2014-01-01-01-00";
-    public static final String OPERATION = "GENERATE";
 
     public static final String CLUSTER_ENTITY_NAME = "primary-cluster";
+    public static final String BCP_CLUSTER_ENTITY_NAME = "bcp-cluster";
     public static final String PROCESS_ENTITY_NAME = "sample-process";
     public static final String COLO_NAME = "west-coast";
-    public static final String WORKFLOW_NAME = "imp-click-join-workflow";
+    public static final String GENERATE_WORKFLOW_NAME = "imp-click-join-workflow";
+    public static final String REPLICATION_WORKFLOW_NAME = "replication-policy-workflow";
     public static final String WORKFLOW_VERSION = "1.0.9";
 
     public static final String INPUT_FEED_NAMES = "impression-feed#clicks-feed";
@@ -82,6 +84,8 @@ public class MetadataMappingServiceTest {
     public static final String OUTPUT_FEED_NAMES = "imp-click-join1,imp-click-join2";
     public static final String OUTPUT_INSTANCE_PATHS =
         "jail://global:00/falcon/imp-click-join1/20140101,jail://global:00/falcon/imp-click-join2/20140101";
+    private static final String REPLICATED_OUTPUT_INSTANCE_PATHS =
+            "jail://global:00/falcon/imp-click-join1/20140101";
 
     public static final String BROKER = "org.apache.activemq.ActiveMQConnectionFactory";
 
@@ -89,7 +93,7 @@ public class MetadataMappingServiceTest {
     private MetadataMappingService service;
 
     private Cluster clusterEntity;
-    private Cluster bcpCluster;
+    private Cluster anotherCluster;
     private List<Feed> inputFeeds = new ArrayList<Feed>();
     private List<Feed> outputFeeds = new ArrayList<Feed>();
     private Process processEntity;
@@ -117,9 +121,7 @@ public class MetadataMappingServiceTest {
     public void tearDown() throws Exception {
         GraphUtils.dump(service.getGraph(), System.out);
 
-        cleanupGraphStore(service.getGraph());
-        cleanupConfigurationStore(configStore);
-        service.destroy();
+        cleanUp();
         StartupProperties.get().setProperty("falcon.graph.preserve.history", "false");
     }
 
@@ -139,9 +141,8 @@ public class MetadataMappingServiceTest {
 
     @Test
     public void testOnAddClusterEntity() throws Exception {
-        clusterEntity = EntityBuilderTestUtil.buildCluster(CLUSTER_ENTITY_NAME, COLO_NAME,
+        clusterEntity = addClusterEntity(CLUSTER_ENTITY_NAME, COLO_NAME,
                 "classification=production");
-        configStore.publish(EntityType.CLUSTER, clusterEntity);
 
         verifyEntityWasAddedToGraph(CLUSTER_ENTITY_NAME, RelationshipType.CLUSTER_ENTITY);
         verifyClusterEntityEdges();
@@ -152,39 +153,35 @@ public class MetadataMappingServiceTest {
 
     @Test (dependsOnMethods = "testOnAddClusterEntity")
     public void testOnAddFeedEntity() throws Exception {
-        Feed impressionsFeed = EntityBuilderTestUtil.buildFeed("impression-feed", clusterEntity,
-                "classified-as=Secure", "analytics");
-        addStorage(impressionsFeed, Storage.TYPE.FILESYSTEM, "/falcon/impression-feed/${YEAR}/${MONTH}/${DAY}");
-        configStore.publish(EntityType.FEED, impressionsFeed);
+        Feed impressionsFeed = addFeedEntity("impression-feed", clusterEntity,
+                "classified-as=Secure", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/impression-feed/${YEAR}/${MONTH}/${DAY}");
         inputFeeds.add(impressionsFeed);
         verifyEntityWasAddedToGraph(impressionsFeed.getName(), RelationshipType.FEED_ENTITY);
         verifyFeedEntityEdges(impressionsFeed.getName());
         Assert.assertEquals(getVerticesCount(service.getGraph()), 7); // +4 = feed, tag, group, user
         Assert.assertEquals(getEdgesCount(service.getGraph()), 6); // +4 = cluster, tag, group, user
 
-        Feed clicksFeed = EntityBuilderTestUtil.buildFeed("clicks-feed", clusterEntity,
-                "classified-as=Secure,classified-as=Financial", "analytics");
-        addStorage(clicksFeed, Storage.TYPE.FILESYSTEM, "/falcon/clicks-feed/${YEAR}-${MONTH}-${DAY}");
-        configStore.publish(EntityType.FEED, clicksFeed);
+        Feed clicksFeed = addFeedEntity("clicks-feed", clusterEntity,
+                "classified-as=Secure,classified-as=Financial", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/clicks-feed/${YEAR}-${MONTH}-${DAY}");
         inputFeeds.add(clicksFeed);
         verifyEntityWasAddedToGraph(clicksFeed.getName(), RelationshipType.FEED_ENTITY);
         Assert.assertEquals(getVerticesCount(service.getGraph()), 9); // feed and financial vertex
         Assert.assertEquals(getEdgesCount(service.getGraph()), 11); // +5 = cluster + user + 2Group + Tag
 
-        Feed join1Feed = EntityBuilderTestUtil.buildFeed("imp-click-join1", clusterEntity,
-                "classified-as=Financial", "reporting,bi");
-        addStorage(join1Feed, Storage.TYPE.FILESYSTEM, "/falcon/imp-click-join1/${YEAR}${MONTH}${DAY}");
-        configStore.publish(EntityType.FEED, join1Feed);
+        Feed join1Feed = addFeedEntity("imp-click-join1", clusterEntity,
+                "classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
+                "/falcon/imp-click-join1/${YEAR}${MONTH}${DAY}");
         outputFeeds.add(join1Feed);
         verifyEntityWasAddedToGraph(join1Feed.getName(), RelationshipType.FEED_ENTITY);
         Assert.assertEquals(getVerticesCount(service.getGraph()), 12); // + 3 = 1 feed and 2 groups
         Assert.assertEquals(getEdgesCount(service.getGraph()), 16); // +5 = cluster + user +
         // Group + 2Tags
 
-        Feed join2Feed = EntityBuilderTestUtil.buildFeed("imp-click-join2", clusterEntity,
-                "classified-as=Secure,classified-as=Financial", "reporting,bi");
-        addStorage(join2Feed, Storage.TYPE.FILESYSTEM, "/falcon/imp-click-join2/${YEAR}${MONTH}${DAY}");
-        configStore.publish(EntityType.FEED, join2Feed);
+        Feed join2Feed = addFeedEntity("imp-click-join2", clusterEntity,
+                "classified-as=Secure,classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
+                "/falcon/imp-click-join2/${YEAR}${MONTH}${DAY}");
         outputFeeds.add(join2Feed);
         verifyEntityWasAddedToGraph(join2Feed.getName(), RelationshipType.FEED_ENTITY);
 
@@ -195,19 +192,9 @@ public class MetadataMappingServiceTest {
 
     @Test (dependsOnMethods = "testOnAddFeedEntity")
     public void testOnAddProcessEntity() throws Exception {
-        processEntity = EntityBuilderTestUtil.buildProcess(PROCESS_ENTITY_NAME, clusterEntity,
-                "classified-as=Critical", "testPipeline,dataReplication_Pipeline");
-        EntityBuilderTestUtil.addProcessWorkflow(processEntity, WORKFLOW_NAME, WORKFLOW_VERSION);
-
-        for (Feed inputFeed : inputFeeds) {
-            EntityBuilderTestUtil.addInput(processEntity, inputFeed);
-        }
-
-        for (Feed outputFeed : outputFeeds) {
-            EntityBuilderTestUtil.addOutput(processEntity, outputFeed);
-        }
-
-        configStore.publish(EntityType.PROCESS, processEntity);
+        processEntity = addProcessEntity(PROCESS_ENTITY_NAME, clusterEntity,
+                "classified-as=Critical", "testPipeline,dataReplication_Pipeline", GENERATE_WORKFLOW_NAME,
+                WORKFLOW_VERSION);
 
         verifyEntityWasAddedToGraph(processEntity.getName(), RelationshipType.PROCESS_ENTITY);
         verifyProcessEntityEdges();
@@ -223,14 +210,13 @@ public class MetadataMappingServiceTest {
         verifyEntityGraph(RelationshipType.FEED_ENTITY, "Secure");
     }
 
-    @Test(dependsOnMethods = "testOnAdd")
+    @Test
     public void testMapLineage() throws Exception {
-        // shutdown the graph and resurrect for testing
-        service.destroy();
-        service.init();
+        setup();
 
-        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(),
-                WorkflowExecutionContext.Type.POST_PROCESSING);
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(
+                EntityOperations.GENERATE, GENERATE_WORKFLOW_NAME, null, null, null, null)
+                , WorkflowExecutionContext.Type.POST_PROCESSING);
         service.onSuccess(context);
 
         debug(service.getGraph());
@@ -243,21 +229,44 @@ public class MetadataMappingServiceTest {
         Assert.assertEquals(getEdgesCount(service.getGraph()), 71);
     }
 
-    @Test (dependsOnMethods = "testMapLineage")
+    @Test
+    public void  testLineageForReplication() throws Exception {
+        setupForLineageReplication();
+
+        String feedName = "imp-click-join1";
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(
+            EntityOperations.REPLICATE, REPLICATION_WORKFLOW_NAME, feedName,
+            REPLICATED_OUTPUT_INSTANCE_PATHS, null, null), WorkflowExecutionContext.Type.POST_PROCESSING);
+        service.onSuccess(context);
+
+        debug(service.getGraph());
+        GraphUtils.dump(service.getGraph());
+        verifyLineageGraph(RelationshipType.FEED_INSTANCE.getName());
+
+        verifyLineageGraphForReplicationOrEviction(feedName, REPLICATED_OUTPUT_INSTANCE_PATHS, context,
+                RelationshipLabel.FEED_CLUSTER_REPLICATED_EDGE);
+
+        // +3 = cluster, colo, tag
+        Assert.assertEquals(getVerticesCount(service.getGraph()), 26);
+        // +3 = +2 edges for bcp cluster, no user but only to colo and new tag  + 1 for replicated-to edge to target
+        // cluster for each output feed instance
+        Assert.assertEquals(getEdgesCount(service.getGraph()), 74);
+    }
+
+    @Test (dependsOnMethods = "testOnAdd")
     public void testOnChange() throws Exception {
         // shutdown the graph and resurrect for testing
         service.destroy();
         service.init();
 
         // cannot modify cluster, adding a new cluster
-        bcpCluster = EntityBuilderTestUtil.buildCluster("bcp-cluster", "east-coast",
-                "classification=bcp");
-        configStore.publish(EntityType.CLUSTER, bcpCluster);
-        verifyEntityWasAddedToGraph("bcp-cluster", RelationshipType.CLUSTER_ENTITY);
+        anotherCluster = addClusterEntity("another-cluster", "east-coast",
+                "classification=another");
+        verifyEntityWasAddedToGraph("another-cluster", RelationshipType.CLUSTER_ENTITY);
 
-        Assert.assertEquals(getVerticesCount(service.getGraph()), 26); // +3 = cluster, colo, tag, 2 pipelines
-        // +4 edges to above, no user but only to colo, new tag, and 2 new pipelines
-        Assert.assertEquals(getEdgesCount(service.getGraph()), 73);
+        Assert.assertEquals(getVerticesCount(service.getGraph()), 20); // +3 = cluster, colo, tag
+        // +2 edges to above, no user but only to colo and new tag
+        Assert.assertEquals(getEdgesCount(service.getGraph()), 33);
     }
 
     @Test(dependsOnMethods = "testOnChange")
@@ -274,7 +283,7 @@ public class MetadataMappingServiceTest {
             // add cluster
             org.apache.falcon.entity.v0.feed.Cluster feedCluster =
                     new org.apache.falcon.entity.v0.feed.Cluster();
-            feedCluster.setName(bcpCluster.getName());
+            feedCluster.setName(anotherCluster.getName());
             newFeed.getClusters().getClusters().add(feedCluster);
 
             configStore.update(EntityType.FEED, newFeed);
@@ -283,8 +292,8 @@ public class MetadataMappingServiceTest {
         }
 
         verifyUpdatedEdges(newFeed);
-        Assert.assertEquals(getVerticesCount(service.getGraph()), 28); //+2 = 2 new tags
-        Assert.assertEquals(getEdgesCount(service.getGraph()), 75); // +2 = 1 new cluster, 1 new tag
+        Assert.assertEquals(getVerticesCount(service.getGraph()), 22); //+2 = 2 new tags
+        Assert.assertEquals(getEdgesCount(service.getGraph()), 35); // +2 = 1 new cluster, 1 new tag
     }
 
     private void verifyUpdatedEdges(Feed newFeed) {
@@ -305,16 +314,16 @@ public class MetadataMappingServiceTest {
         for (Edge clusterEdge : feedVertex.getEdges(Direction.OUT, RelationshipLabel.FEED_CLUSTER_EDGE.getName())) {
             actual.add(clusterEdge.getVertex(Direction.IN).<String>getProperty("name"));
         }
-        Assert.assertTrue(actual.containsAll(Arrays.asList("primary-cluster", "bcp-cluster")),
+        Assert.assertTrue(actual.containsAll(Arrays.asList("primary-cluster", "another-cluster")),
                 "Actual does not contain expected: " + actual);
     }
 
     @Test(dependsOnMethods = "testOnFeedEntityChange")
     public void testOnProcessEntityChange() throws Exception {
         Process oldProcess = processEntity;
-        Process newProcess = EntityBuilderTestUtil.buildProcess(oldProcess.getName(), bcpCluster,
+        Process newProcess = EntityBuilderTestUtil.buildProcess(oldProcess.getName(), anotherCluster,
                 null, null);
-        EntityBuilderTestUtil.addProcessWorkflow(newProcess, WORKFLOW_NAME, "2.0.0");
+        EntityBuilderTestUtil.addProcessWorkflow(newProcess, GENERATE_WORKFLOW_NAME, "2.0.0");
         EntityBuilderTestUtil.addInput(newProcess, inputFeeds.get(0));
 
         try {
@@ -325,8 +334,8 @@ public class MetadataMappingServiceTest {
         }
 
         verifyUpdatedEdges(newProcess);
-        Assert.assertEquals(getVerticesCount(service.getGraph()), 28); // +0, no net new
-        Assert.assertEquals(getEdgesCount(service.getGraph()), 69); // -6 = -2 outputs, -1 tag, -1 cluster, -2 pipelines
+        Assert.assertEquals(getVerticesCount(service.getGraph()), 22); // +0, no net new
+        Assert.assertEquals(getEdgesCount(service.getGraph()), 29); // -6 = -2 outputs, -1 tag, -1 cluster, -2 pipelines
     }
 
     @Test(dependsOnMethods = "testOnProcessEntityChange")
@@ -366,7 +375,7 @@ public class MetadataMappingServiceTest {
         // cluster
         Edge edge = processVertex.getEdges(Direction.OUT,
                 RelationshipLabel.PROCESS_CLUSTER_EDGE.getName()).iterator().next();
-        Assert.assertEquals(edge.getVertex(Direction.IN).getProperty("name"), bcpCluster.getName());
+        Assert.assertEquals(edge.getVertex(Direction.IN).getProperty("name"), anotherCluster.getName());
 
         // inputs
         edge = processVertex.getEdges(Direction.IN, RelationshipLabel.FEED_PROCESS_EDGE.getName()).iterator().next();
@@ -391,6 +400,40 @@ public class MetadataMappingServiceTest {
         }
     }
 
+    private Cluster addClusterEntity(String name, String colo, String tags) throws Exception {
+        Cluster cluster = EntityBuilderTestUtil.buildCluster(name, colo, tags);
+        configStore.publish(EntityType.CLUSTER, cluster);
+        return cluster;
+    }
+
+    private Feed addFeedEntity(String feedName, Cluster cluster, String tags, String groups,
+                              Storage.TYPE storageType, String uriTemplate) throws Exception {
+        Feed feed = EntityBuilderTestUtil.buildFeed(feedName, cluster,
+                tags, groups);
+        addStorage(feed, storageType, uriTemplate);
+        configStore.publish(EntityType.FEED, feed);
+        return feed;
+    }
+
+    public Process addProcessEntity(String processName, Cluster cluster,
+                                    String tags, String pipelineTags, String workflowName,
+                                    String version) throws Exception {
+        Process process = EntityBuilderTestUtil.buildProcess(processName, cluster,
+                tags, pipelineTags);
+        EntityBuilderTestUtil.addProcessWorkflow(process, workflowName, version);
+
+        for (Feed inputFeed : inputFeeds) {
+            EntityBuilderTestUtil.addInput(process, inputFeed);
+        }
+
+        for (Feed outputFeed : outputFeeds) {
+            EntityBuilderTestUtil.addOutput(process, outputFeed);
+        }
+
+        configStore.publish(EntityType.PROCESS, process);
+        return process;
+    }
+
     private static void addStorage(Feed feed, Storage.TYPE storageType, String uriTemplate) {
         if (storageType == Storage.TYPE.FILESYSTEM) {
             Locations locations = new Locations();
@@ -633,19 +676,49 @@ public class MetadataMappingServiceTest {
                         "imp-click-join1/2014-01-01T00:00Z", "imp-click-join2/2014-01-01T00:00Z"));
     }
 
-    private static String[] getTestMessageArgs() {
+    private void verifyLineageGraphForReplicationOrEviction(String feedName, String feedInstanceDataPath,
+                                                            WorkflowExecutionContext context,
+                                                            RelationshipLabel edgeLabel) throws Exception {
+        String feedInstanceName = InstanceRelationshipGraphBuilder.getFeedInstanceName(feedName
+                , context.getSrcClusterName(), feedInstanceDataPath);
+        Vertex feedVertex = getEntityVertex(feedInstanceName, RelationshipType.FEED_INSTANCE);
+
+        Edge edge = feedVertex.getEdges(Direction.OUT, edgeLabel.getName())
+                .iterator().next();
+        Assert.assertNotNull(edge);
+        Assert.assertEquals(edge.getProperty(RelationshipProperty.TIMESTAMP.getName())
+                , context.getTimeStampAsISO8601());
+
+        Vertex clusterVertex = edge.getVertex(Direction.IN);
+        Assert.assertEquals(clusterVertex.getProperty(RelationshipProperty.NAME.getName()), context.getClusterName());
+    }
+
+    private static String[] getTestMessageArgs(EntityOperations operation, String wfName, String outputFeedNames,
+                                               String feedInstancePaths, String falconInputPaths,
+                                               String falconInputFeeds) {
+        String cluster;
+        if (EntityOperations.REPLICATE == operation) {
+            cluster = BCP_CLUSTER_ENTITY_NAME + WorkflowExecutionContext.CLUSTER_NAME_SEPARATOR + CLUSTER_ENTITY_NAME;
+        } else {
+            cluster = CLUSTER_ENTITY_NAME;
+        }
+
         return new String[]{
-            "-" + WorkflowExecutionArgs.CLUSTER_NAME.getName(), CLUSTER_ENTITY_NAME,
+            "-" + WorkflowExecutionArgs.CLUSTER_NAME.getName(), cluster,
             "-" + WorkflowExecutionArgs.ENTITY_TYPE.getName(), ("process"),
             "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), PROCESS_ENTITY_NAME,
             "-" + WorkflowExecutionArgs.NOMINAL_TIME.getName(), NOMINAL_TIME,
-            "-" + WorkflowExecutionArgs.OPERATION.getName(), OPERATION,
+            "-" + WorkflowExecutionArgs.OPERATION.getName(), operation.toString(),
 
-            "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), INPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(), INPUT_INSTANCE_PATHS,
+            "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(),
+            (falconInputFeeds != null ? falconInputFeeds : INPUT_FEED_NAMES),
+            "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(),
+            (falconInputPaths != null ? falconInputPaths : INPUT_INSTANCE_PATHS),
 
-            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(), OUTPUT_INSTANCE_PATHS,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(),
+            (outputFeedNames != null ? outputFeedNames : OUTPUT_FEED_NAMES),
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
+            (feedInstancePaths != null ? feedInstancePaths : OUTPUT_INSTANCE_PATHS),
 
             "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
             "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), FALCON_USER,
@@ -655,11 +728,10 @@ public class MetadataMappingServiceTest {
 
             "-" + WorkflowExecutionArgs.WF_ENGINE_URL.getName(), "http://localhost:11000/oozie",
             "-" + WorkflowExecutionArgs.USER_SUBFLOW_ID.getName(), "userflow@wf-id",
-            "-" + WorkflowExecutionArgs.USER_WORKFLOW_NAME.getName(), WORKFLOW_NAME,
+            "-" + WorkflowExecutionArgs.USER_WORKFLOW_NAME.getName(), wfName,
             "-" + WorkflowExecutionArgs.USER_WORKFLOW_VERSION.getName(), WORKFLOW_VERSION,
             "-" + WorkflowExecutionArgs.USER_WORKFLOW_ENGINE.getName(), EngineType.PIG.name(),
 
-
             "-" + WorkflowExecutionArgs.BRKR_IMPL_CLASS.getName(), BROKER,
             "-" + WorkflowExecutionArgs.BRKR_URL.getName(), "tcp://localhost:61616?daemon=true",
             "-" + WorkflowExecutionArgs.USER_BRKR_IMPL_CLASS.getName(), BROKER,
@@ -671,6 +743,54 @@ public class MetadataMappingServiceTest {
         };
     }
 
+    private void setup() throws Exception {
+        cleanUp();
+        service.init();
+
+        // Add cluster
+        clusterEntity = addClusterEntity(CLUSTER_ENTITY_NAME, COLO_NAME,
+                "classification=production");
+
+        // Add input and output feeds
+        Feed impressionsFeed = addFeedEntity("impression-feed", clusterEntity,
+                "classified-as=Secure", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/impression-feed/${YEAR}/${MONTH}/${DAY}");
+        inputFeeds.add(impressionsFeed);
+        Feed clicksFeed = addFeedEntity("clicks-feed", clusterEntity,
+                "classified-as=Secure,classified-as=Financial", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/clicks-feed/${YEAR}-${MONTH}-${DAY}");
+        inputFeeds.add(clicksFeed);
+        Feed join1Feed = addFeedEntity("imp-click-join1", clusterEntity,
+                "classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
+                "/falcon/imp-click-join1/${YEAR}${MONTH}${DAY}");
+        outputFeeds.add(join1Feed);
+        Feed join2Feed = addFeedEntity("imp-click-join2", clusterEntity,
+                "classified-as=Secure,classified-as=Financial", "reporting,bi", Storage.TYPE.FILESYSTEM,
+                "/falcon/imp-click-join2/${YEAR}${MONTH}${DAY}");
+        outputFeeds.add(join2Feed);
+        processEntity = addProcessEntity(PROCESS_ENTITY_NAME, clusterEntity,
+                "classified-as=Critical", "testPipeline,dataReplication_Pipeline", GENERATE_WORKFLOW_NAME,
+                WORKFLOW_VERSION);
+
+    }
+
+    private void setupForLineageReplication() throws Exception {
+        setup();
+        // GENERATE WF should have run before this to create all instance related vertices
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(getTestMessageArgs(
+            EntityOperations.GENERATE, GENERATE_WORKFLOW_NAME, null, null, null, null)
+            , WorkflowExecutionContext.Type.POST_PROCESSING);
+        service.onSuccess(context);
+        // Add backup cluster
+        addClusterEntity(BCP_CLUSTER_ENTITY_NAME, "east-coast", "classification=bcp");
+    }
+
+    private void cleanUp() throws Exception {
+        cleanupGraphStore(service.getGraph());
+        cleanupConfigurationStore(configStore);
+        service.destroy();
+    }
+
     private void cleanupGraphStore(Graph graph) {
         for (Edge edge : graph.getEdges()) {
             graph.removeEdge(edge);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
index 966f90e..5697eb6 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
@@ -159,6 +159,10 @@ public class FeedReplicationCoordinatorBuilder extends OozieCoordinatorBuilder<F
 
         workflow.setAppPath(getStoragePath(buildPath));
         Properties props = createCoordDefaultConfiguration(trgCluster, wfName);
+        // Override CLUSTER_NAME property to include both source and target cluster
+        String clusterProperty = trgCluster.getName()
+                + WorkflowExecutionContext.CLUSTER_NAME_SEPARATOR + srcCluster.getName();
+        props.put(WorkflowExecutionArgs.CLUSTER_NAME.getName(), clusterProperty);
         props.put("srcClusterName", srcCluster.getName());
         props.put("srcClusterColo", srcCluster.getColo());
         if (props.get(MR_MAX_MAPS) == null) { // set default if user has not overridden

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java b/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
index 3c49353..379cf34 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
@@ -192,7 +192,7 @@ public class OozieFeedWorkflowBuilderTest extends AbstractTestBase {
 
         HashMap<String, String> props = getCoordProperties(coord);
 
-        verifyEntityProperties(feed, trgCluster,
+        verifyEntityProperties(feed, trgCluster, srcCluster,
                 WorkflowExecutionContext.EntityOperations.REPLICATE, props);
         verifyBrokerProperties(trgCluster, props);
 
@@ -332,7 +332,7 @@ public class OozieFeedWorkflowBuilderTest extends AbstractTestBase {
         Assert.assertEquals(props.get("maxMaps"), "33");
         Assert.assertEquals(props.get("mapBandwidthKB"), "2048");
 
-        verifyEntityProperties(aFeed, aCluster,
+        verifyEntityProperties(aFeed, aCluster, srcCluster,
                 WorkflowExecutionContext.EntityOperations.REPLICATE, props);
         verifyBrokerProperties(trgCluster, props);
     }
@@ -456,7 +456,7 @@ public class OozieFeedWorkflowBuilderTest extends AbstractTestBase {
         assertReplicationHCatCredentials(getWorkflowapp(trgMiniDFS.getFileSystem(), coord),
                 wfPath.toString());
 
-        verifyEntityProperties(tableFeed, trgCluster,
+        verifyEntityProperties(tableFeed, trgCluster, srcCluster,
                 WorkflowExecutionContext.EntityOperations.REPLICATE, props);
         verifyBrokerProperties(trgCluster, props);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/23eed9f6/oozie/src/test/java/org/apache/falcon/oozie/process/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/process/AbstractTestBase.java b/oozie/src/test/java/org/apache/falcon/oozie/process/AbstractTestBase.java
index b547c31..4e260e9 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/process/AbstractTestBase.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/process/AbstractTestBase.java
@@ -217,11 +217,22 @@ public class AbstractTestBase {
     protected void verifyEntityProperties(Entity entity, Cluster cluster,
                                           WorkflowExecutionContext.EntityOperations operation,
                                           HashMap<String, String> props) throws Exception {
+        verifyEntityProperties(entity, cluster, null, operation, props);
+    }
+
+    protected void verifyEntityProperties(Entity entity, Cluster cluster, Cluster srcCluster,
+                                          WorkflowExecutionContext.EntityOperations operation,
+                                          HashMap<String, String> props) throws Exception {
         Assert.assertEquals(props.get(WorkflowExecutionArgs.ENTITY_NAME.getName()),
                 entity.getName());
         Assert.assertEquals(props.get(WorkflowExecutionArgs.ENTITY_TYPE.getName()),
                 entity.getEntityType().name());
-        Assert.assertEquals(props.get(WorkflowExecutionArgs.CLUSTER_NAME.getName()), cluster.getName());
+        if (WorkflowExecutionContext.EntityOperations.REPLICATE == operation) {
+            Assert.assertEquals(props.get(WorkflowExecutionArgs.CLUSTER_NAME.getName()),
+                    cluster.getName() + WorkflowExecutionContext.CLUSTER_NAME_SEPARATOR + srcCluster.getName());
+        } else {
+            Assert.assertEquals(props.get(WorkflowExecutionArgs.CLUSTER_NAME.getName()), cluster.getName());
+        }
         Assert.assertEquals(props.get(WorkflowExecutionArgs.LOG_DIR.getName()), getLogPath(cluster, entity));
         Assert.assertEquals(props.get("falconDataOperation"), operation.name());
     }


[34/41] git commit: FALCON-656 add test in falcon regression's Authorization test where non-feed owner updates a feed with a dependent process contributed by Karishma

Posted by ra...@apache.org.
FALCON-656 add test in falcon regression's Authorization test where non-feed owner updates a feed with a dependent process contributed by Karishma


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

Branch: refs/heads/FALCON-585
Commit: 245b6b45755c375e3d5af4b113a3b206d1669727
Parents: 120c1c0
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Mon Sep 8 15:24:39 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Mon Sep 8 15:24:39 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  2 ++
 .../falcon/regression/AuthorizationTest.java    | 28 ++++++++++++++++++++
 2 files changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/245b6b45/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index cc38357..231f33e 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,6 +9,8 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
+   FALCON-656 add test in falcon regression's Authorization test where non-feed owner updates
+   a feed with a dependent process(Karishma via Samarth Gupta)
    FALCON-674 General code factored out for ProcessInstance* tests (Paul Isaychuk via Ruslan
    Ostafiychuk)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/245b6b45/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
index 8bf4288..49b13b5 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
@@ -669,6 +669,34 @@ public class AuthorizationTest extends BaseTestClass {
         Assert.assertEquals(oldProcessUser, newProcessUser, "User should be the same");
     }
 
+    public void u1ScheduleFeedU2ScheduleDependantProcessU2UpdateFeed() throws Exception {
+        String feed = bundles[0].getInputFeedFromBundle();
+        String process = bundles[0].getProcessData();
+        //submit both feeds
+        bundles[0].submitClusters(prism);
+        bundles[0].submitFeeds(prism);
+        //schedule input feed by U1
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(Util.URLS.SCHEDULE_URL, feed));
+        AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
+
+        //by U2 schedule process dependent on scheduled feed by U1
+        KerberosHelper.loginFromKeytab(MerlinConstants.USER2_NAME);
+        ServiceResponse serviceResponse = prism.getProcessHelper().submitAndSchedule(Util
+                 .URLS.SUBMIT_AND_SCHEDULE_URL, process, MerlinConstants.USER2_NAME);
+        AssertUtil.assertSucceeded(serviceResponse);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+
+        //update feed definition
+        String newFeed = Util.setFeedPathValue(feed,
+        baseHDFSDir + "/randomPath/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/");
+
+        //update feed by U2
+        serviceResponse = prism.getFeedHelper().update(feed, newFeed,
+        TimeUtil.getTimeWrtSystemTime(0), MerlinConstants.USER2_NAME);
+        AssertUtil.assertFailedWithStatus(serviceResponse, HttpStatus.SC_BAD_REQUEST,
+                 "Feed scheduled by first user should not be updated by second user");
+    }
+
     private String getBundleUser(ColoHelper coloHelper, String entityName, EntityType entityType)
         throws OozieClientException {
         String newProcessBundleId = InstanceUtil.getLatestBundleID(coloHelper, entityName,


[15/41] git commit: FALCON-655 Skip workflow upload if process won't be submitted

Posted by ra...@apache.org.
FALCON-655 Skip workflow upload if process won't be submitted


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

Branch: refs/heads/FALCON-585
Commit: d74dc3221f25ff99156a6b2a0aa1e8cd94d6c0f9
Parents: fad72d2
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Mon Sep 1 14:42:18 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Mon Sep 1 15:04:08 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                                | 2 ++
 .../java/org/apache/falcon/regression/FeedResumeTest.java    | 2 --
 .../java/org/apache/falcon/regression/FeedScheduleTest.java  | 7 -------
 .../java/org/apache/falcon/regression/FeedStatusTest.java    | 6 ------
 .../apache/falcon/regression/FeedSubmitAndScheduleTest.java  | 8 --------
 .../java/org/apache/falcon/regression/FeedSubmitTest.java    | 7 -------
 .../java/org/apache/falcon/regression/FeedSuspendTest.java   | 7 -------
 7 files changed, 2 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 57091a4..72daf8e 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -10,6 +10,8 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+   FALCON-655 Skip workflow upload if process won't be submitted (Ruslan Ostafiychuk)
+
    FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if
    clean_test_dir=false (Ruslan Ostafiychuk)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
index 2f6653a..a3f37ab 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
@@ -50,7 +50,6 @@ public class FeedResumeTest extends BaseTestClass {
     private String feed;
     private ColoHelper cluster = servers.get(0);
     private OozieClient clusterOC = serverOC.get(0);
-    private String aggregateWorkflowDir = baseHDFSDir + "/FeedResumeTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedResumeTest.class);
 
     @BeforeMethod(alwaysRun = true)
@@ -59,7 +58,6 @@ public class FeedResumeTest extends BaseTestClass {
         bundles[0] = BundleUtil.readELBundle();
         bundles[0].generateUniqueBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         bundles[0].submitClusters(prism);
         feed = bundles[0].getInputFeedFromBundle();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
index 8d08f87..5d87e6e 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
@@ -24,7 +24,6 @@ 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -47,20 +46,14 @@ public class FeedScheduleTest extends BaseTestClass {
     private ColoHelper cluster = servers.get(0);
     private OozieClient clusterOC = serverOC.get(0);
     private String feed;
-    private String aggregateWorkflowDir = baseHDFSDir + "/FeedScheduleTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedScheduleTest.class);
 
-    public void uploadWorkflow() throws Exception {
-        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-    }
-
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
         bundles[0] = BundleUtil.readELBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         Bundle.submitCluster(bundles[0]);
         feed = bundles[0].getInputFeedFromBundle();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
index 2fa4ed5..8ce88d4 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
@@ -25,7 +25,6 @@ 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.Util;
 import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
@@ -50,12 +49,8 @@ public class FeedStatusTest extends BaseTestClass {
     private ColoHelper cluster = servers.get(0);
     private OozieClient clusterOC = serverOC.get(0);
     private String feed;
-    private String aggregateWorkflowDir = baseHDFSDir + "/FeedStatusTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedStatusTest.class);
 
-    public void uploadWorkflow() throws Exception {
-        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-    }
 
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
@@ -63,7 +58,6 @@ public class FeedStatusTest extends BaseTestClass {
         bundles[0] = BundleUtil.readELBundle();
         bundles[0].generateUniqueBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
 
         //submit the cluster
         ServiceResponse response =

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
index 442a5b2..1fdfa95 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
@@ -25,7 +25,6 @@ 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.InstanceUtil;
-import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.Util;
 import org.apache.falcon.regression.core.util.Util.URLS;
@@ -53,22 +52,15 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
 
     private ColoHelper cluster = servers.get(0);
     private OozieClient clusterOC = serverOC.get(0);
-    private String aggregateWorkflowDir = baseHDFSDir + "/FeedSubmitAndScheduleTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedSubmitAndScheduleTest.class);
     private String feed;
 
     @BeforeMethod(alwaysRun = true)
-    public void uploadWorkflow() throws Exception {
-        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-    }
-
-    @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
         bundles[0] = BundleUtil.readELBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
         bundles[0].generateUniqueBundle();
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
         feed = bundles[0].getDataSets().get(0);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
index 1fc8530..72f4921 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
@@ -24,7 +24,6 @@ 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -44,20 +43,14 @@ public class FeedSubmitTest extends BaseTestClass {
 
     private ColoHelper cluster = servers.get(0);
     private String feed;
-    private String aggregateWorkflowDir = baseHDFSDir + "/FeedSubmitTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedSubmitTest.class);
 
-    public void uploadWorkflow() throws Exception {
-        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-    }
-
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
         bundles[0] = BundleUtil.readELBundle();
         bundles[0].generateUniqueBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
 
         //submit the cluster
         ServiceResponse response =

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/d74dc322/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
index d1735a0..1973bf5 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
@@ -24,7 +24,6 @@ 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -47,20 +46,14 @@ public class FeedSuspendTest extends BaseTestClass {
     private ColoHelper cluster = servers.get(0);
     private OozieClient clusterOC = serverOC.get(0);
     private String feed;
-    private String aggregateWorkflowDir = baseHDFSDir + "/FeedSuspendTest/aggregator";
     private static final Logger LOGGER = Logger.getLogger(FeedSuspendTest.class);
 
-    public void uploadWorkflow() throws Exception {
-        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-    }
-
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
         LOGGER.info("test name: " + method.getName());
         bundles[0] = BundleUtil.readELBundle();
         bundles[0].generateUniqueBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
 
         //submit the cluster
         ServiceResponse response =


[14/41] git commit: FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if clean_test_dir=false

Posted by ra...@apache.org.
FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if clean_test_dir=false


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

Branch: refs/heads/FALCON-585
Commit: fad72d2ab2b2e470ccd7f83de87993fb351e3305
Parents: 9774414
Author: Ruslan Ostafiychuk <ro...@apache.org>
Authored: Mon Sep 1 09:22:39 2014 +0300
Committer: Ruslan Ostafiychuk <ro...@apache.org>
Committed: Mon Sep 1 09:22:39 2014 +0300

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  3 ++
 .../core/enumsAndConstants/MerlinConstants.java |  3 ++
 .../regression/testHelper/BaseTestClass.java    |  8 +++++
 .../falcon/regression/AuthorizationTest.java    |  6 ++++
 .../regression/ELExp_FutureAndLatestTest.java   | 12 +++-----
 .../falcon/regression/ELValidationsTest.java    |  5 +++
 .../regression/EmbeddedPigScriptTest.java       | 12 +++-----
 .../regression/FeedClusterUpdateTest.java       |  7 +++++
 .../regression/FeedInstanceStatusTest.java      |  9 ++++--
 .../falcon/regression/FeedReplicationTest.java  |  5 +++
 .../falcon/regression/FeedResumeTest.java       |  8 ++++-
 .../falcon/regression/FeedScheduleTest.java     |  7 +++++
 .../falcon/regression/FeedStatusTest.java       |  7 +++++
 .../regression/FeedSubmitAndScheduleTest.java   |  9 ++++--
 .../falcon/regression/FeedSubmitTest.java       |  7 +++++
 .../falcon/regression/FeedSuspendTest.java      |  7 +++++
 .../falcon/regression/InstanceParamTest.java    |  6 ++++
 .../falcon/regression/InstanceSummaryTest.java  |  6 ++++
 .../apache/falcon/regression/NewRetryTest.java  |  6 ++++
 .../falcon/regression/NoOutputProcessTest.java  |  6 ++++
 .../ProcessInstanceColoMixedTest.java           |  7 +++++
 .../regression/ProcessInstanceKillsTest.java    | 18 ++++-------
 .../regression/ProcessInstanceRerunTest.java    | 15 +++------
 .../regression/ProcessInstanceResumeTest.java   | 19 ++++--------
 .../regression/ProcessInstanceRunningTest.java  | 16 +++-------
 .../regression/ProcessInstanceStatusTest.java   |  6 ++++
 .../regression/ProcessInstanceSuspendTest.java  | 15 +++------
 .../falcon/regression/ProcessLibPathTest.java   |  7 +++++
 .../regression/hcat/HCatFeedOperationsTest.java |  4 ++-
 .../falcon/regression/hcat/HCatProcessTest.java |  7 +++++
 .../regression/hcat/HCatReplicationTest.java    |  6 ++++
 .../regression/hcat/HCatRetentionTest.java      |  5 +++
 .../lineage/LineageApiProcessInstanceTest.java  |  6 ++++
 .../regression/lineage/LineageApiTest.java      | 10 +++++-
 .../regression/prism/EntityDryRunTest.java      | 32 ++++++++++++++------
 .../prism/FeedDelayParallelTimeoutTest.java     |  7 +++++
 .../regression/prism/FeedRetentionTest.java     |  7 +++++
 .../prism/NewPrismProcessUpdateTest.java        |  5 +++
 .../regression/prism/OptionalInputTest.java     |  7 +++++
 .../prism/PrismClusterDeleteTest.java           |  6 ++++
 .../prism/PrismConcurrentRequestTest.java       |  6 ++++
 .../regression/prism/PrismFeedDeleteTest.java   |  6 ++++
 .../prism/PrismFeedLateReplicationTest.java     |  7 +++++
 .../PrismFeedReplicationPartitionExpTest.java   |  6 ++++
 .../prism/PrismFeedReplicationUpdateTest.java   |  6 ++++
 .../regression/prism/PrismFeedResumeTest.java   |  6 ++++
 .../regression/prism/PrismFeedScheduleTest.java |  6 ++++
 .../regression/prism/PrismFeedSnSTest.java      |  6 ++++
 .../regression/prism/PrismFeedSuspendTest.java  |  6 ++++
 .../regression/prism/PrismFeedUpdateTest.java   |  6 ++++
 .../prism/PrismProcessDeleteTest.java           |  7 ++++-
 .../prism/PrismProcessResumeTest.java           |  7 +++++
 .../prism/PrismProcessScheduleTest.java         |  9 +++++-
 .../regression/prism/PrismProcessSnSTest.java   |  6 ++++
 .../prism/PrismProcessSuspendTest.java          |  7 +++++
 .../regression/prism/PrismSubmitTest.java       |  6 ++++
 .../prism/ProcessPartitionExpVariableTest.java  |  7 ++++-
 .../prism/RescheduleKilledProcessTest.java      |  7 +++++
 .../RescheduleProcessInFinalStatesTest.java     |  6 ++++
 .../falcon/regression/prism/RetentionTest.java  |  5 +++
 .../prism/UpdateAtSpecificTimeTest.java         |  6 ++++
 .../falcon/regression/ui/LineageGraphTest.java  |  1 +
 .../falcon/regression/ui/ProcessUITest.java     |  6 ++++
 63 files changed, 395 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index f5cfb93..57091a4 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -10,6 +10,9 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+   FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if
+   clean_test_dir=false (Ruslan Ostafiychuk)
+
    FALCON-646 Refactoring, documentation stuff (Paul Isaychuk via Ruslan Ostafiychuk)
 
    FALCON-572 HadoopUtil cleanup in falcon-regression (Ruslan Ostafiychuk via Samarth Gupta)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
index 3c6779f..edd75ed 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/enumsAndConstants/MerlinConstants.java
@@ -47,6 +47,9 @@ public final class MerlinConstants {
     public static final String ACL_OWNER = Config.getProperty("ACL.OWNER", RequestKeys.CURRENT_USER);
     public static final String ACL_GROUP = Config.getProperty("ACL.GROUP", "default");
 
+    public static final boolean CLEAN_TEST_DIR =
+        Boolean.valueOf(Config.getProperty("clean_test_dir", "true"));
+
     /* initialize keyTabMap */
     static {
         final String currentUserKeytab = Config.getProperty(CURRENT_USER_KEYTAB_STR);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java b/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
index c64ddad..a951da7 100644
--- a/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
+++ b/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
@@ -111,4 +111,12 @@ public class BaseTestClass {
             }
         }
     }
+
+    public void cleanTestDirs() throws IOException {
+        if (MerlinConstants.CLEAN_TEST_DIR) {
+            for (FileSystem fs : serverFS) {
+                HadoopUtil.deleteDirIfExists(baseHDFSDir, fs);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
index dd901fb..fac215f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
@@ -45,6 +45,7 @@ import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.OozieClientException;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -690,4 +691,9 @@ public class AuthorizationTest extends BaseTestClass {
         KerberosHelper.loginFromKeytab(MerlinConstants.CURRENT_USER_NAME);
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
index dcec150..63484c1 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
@@ -38,6 +38,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -50,7 +51,6 @@ public class ELExp_FutureAndLatestTest extends BaseTestClass {
     ColoHelper cluster = servers.get(0);
     FileSystem clusterFS = serverFS.get(0);
     OozieClient clusterOC = serverOC.get(0);
-    private String prefix;
     private String baseTestDir = baseHDFSDir + "/ELExp_FutureAndLatest";
     private String aggregateWorkflowDir = baseTestDir + "/aggregator";
     private static final Logger logger = Logger.getLogger(ELExp_FutureAndLatestTest.class);
@@ -70,12 +70,11 @@ public class ELExp_FutureAndLatestTest extends BaseTestClass {
         b.setInputFeedDataPath(
             baseTestDir + "/ELExp_latest/testData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
         b.setProcessWorkflow(aggregateWorkflowDir);
-        prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 1);
 
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            b.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -118,8 +117,7 @@ public class ELExp_FutureAndLatestTest extends BaseTestClass {
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        logger.info("in @AfterClass");
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELValidationsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELValidationsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELValidationsTest.java
index 5425360..7555c0c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELValidationsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELValidationsTest.java
@@ -31,6 +31,7 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.TestNGException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -286,4 +287,8 @@ public class ELValidationsTest extends BaseTestClass {
         return df.format(num);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
index 0d89fac..00f68e6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
@@ -50,6 +50,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -62,7 +63,6 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
     ColoHelper cluster = servers.get(0);
     FileSystem clusterFS = serverFS.get(0);
     OozieClient clusterOC = serverOC.get(0);
-    private String prefix;
     String pigTestDir = baseHDFSDir + "/EmbeddedPigScriptTest";
     String pigScriptDir = pigTestDir + "/EmbeddedPigScriptTest/pig";
     String pigScriptLocation = pigScriptDir + "/id.pig";
@@ -84,10 +84,9 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
         String startDate = "2010-01-02T00:40Z";
         String endDate = "2010-01-02T01:10Z";
         bundle.setInputFeedDataPath(inputPath);
-        prefix = bundle.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            bundle.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -181,8 +180,7 @@ public class EmbeddedPigScriptTest extends BaseTestClass {
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        logger.info("in @AfterClass");
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
index b06d8dd..17120e9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
@@ -36,11 +36,13 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /**
@@ -854,4 +856,9 @@ public class FeedClusterUpdateTest extends BaseTestClass {
 
     }
     */
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
index acf3bb3..d4fab30 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
@@ -23,7 +23,6 @@ import org.apache.falcon.entity.v0.feed.ActionType;
 import org.apache.falcon.entity.v0.feed.ClusterType;
 import org.apache.falcon.regression.core.helpers.ColoHelper;
 import org.apache.falcon.regression.core.response.InstancesResult;
-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.HadoopUtil;
@@ -36,12 +35,13 @@ import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
-import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 
@@ -243,4 +243,9 @@ public class FeedInstanceStatusTest extends BaseTestClass {
 
         logger.info(responseInstance.getMessage());
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
index 8b1dd0e..c4c11cf 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
@@ -47,6 +47,7 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -359,4 +360,8 @@ public class FeedReplicationTest extends BaseTestClass {
         AssertUtil.checkForListSizes(cluster1ReplicatedData, cluster2ReplicatedData);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
index 937f0a7..2f6653a 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
@@ -26,17 +26,18 @@ import org.apache.falcon.regression.core.interfaces.IEntityManagerHelper;
 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.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /**
@@ -126,4 +127,9 @@ public class FeedResumeTest extends BaseTestClass {
         Assert.assertTrue(response.getMessage().contains(colo + "/RUNNING"));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
index 148e3bf..8d08f87 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
@@ -30,10 +30,12 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /**
@@ -150,4 +152,9 @@ public class FeedScheduleTest extends BaseTestClass {
     public void scheduleNonExistentFeed() throws Exception {
         AssertUtil.assertFailed(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
index 96af174..2fa4ed5 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
@@ -33,10 +33,12 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /**
@@ -176,4 +178,9 @@ public class FeedStatusTest extends BaseTestClass {
             response.getMessage().contains(Util.readEntityName(feed) + " (FEED) not found"));
 
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
index 38cf080..442a5b2 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
@@ -35,6 +35,7 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -173,7 +174,9 @@ public class FeedSubmitAndScheduleTest extends BaseTestClass {
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
     }
-}
-
-
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
index a50c32a..1fc8530 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
@@ -28,10 +28,12 @@ import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /**
@@ -127,4 +129,9 @@ public class FeedSubmitTest extends BaseTestClass {
         response = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
         AssertUtil.assertSucceeded(response);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
index 10bdddb..d1735a0 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
@@ -30,10 +30,12 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /**
@@ -153,4 +155,9 @@ public class FeedSuspendTest extends BaseTestClass {
         response = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
         AssertUtil.assertFailed(response);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
index ad2e832..d733cfc 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
@@ -37,6 +37,7 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.OozieClientException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -160,4 +161,9 @@ public class InstanceParamTest extends BaseTestClass {
             HadoopUtil.deleteDirIfExists(Util.getPathPrefix(feedInputPath), fs);
         }
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index 9901fb1..154485f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction.Status;
 import org.apache.oozie.client.OozieClientException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -306,4 +307,9 @@ public class InstanceSummaryTest extends BaseTestClass {
         processBundle.deleteBundle(prism);
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
index 5ab3dfe..48bf185 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
@@ -50,12 +50,14 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -1180,5 +1182,9 @@ public class NewRetryTest extends BaseTestClass {
 
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
index df63483..25456a2 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
@@ -35,11 +35,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -145,4 +147,8 @@ public class NoOutputProcessTest extends BaseTestClass {
         Util.printMessageData(consumerProcessMsg);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
index 278d973..25ef675 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
@@ -39,11 +39,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction.Status;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -279,5 +281,10 @@ public class ProcessInstanceColoMixedTest extends BaseTestClass {
         AssertUtil.assertSucceeded(responseInstance);
         Assert.assertTrue(responseInstance.getInstances() != null);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
index e4129e6..33d6b07 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
@@ -41,6 +41,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -72,10 +73,9 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         String startDate = "2010-01-01T23:20Z";
         String endDate = "2010-01-02T01:21Z";
         b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            b.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -180,7 +180,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startTimeData, endTimeData, 1);
         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.SINGLE_FILE,
             baseTestHDFSDir + "/input01", dataDates);
-        bundles[0].setInputFeedDataPath(feedInputPath.replace("input/","input01/"));
+        bundles[0].setInputFeedDataPath(feedInputPath.replace("input/", "input01/"));
         bundles[0].setProcessValidity(startTime, endTime);
         bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
         bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);
@@ -308,13 +308,7 @@ public class ProcessInstanceKillsTest extends BaseTestClass {
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle b = BundleUtil.readELBundle();
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
index df65a79..3160c1d 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
@@ -42,6 +42,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -73,10 +74,9 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
         String startDate = "2010-01-02T00:40Z";
         String endDate = "2010-01-02T01:20Z";
         b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            b.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -290,12 +290,7 @@ public class ProcessInstanceRerunTest extends BaseTestClass {
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle b = BundleUtil.readELBundle();
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
index 246a41e..e324c26 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
@@ -40,6 +40,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -70,11 +71,10 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
         String startDate = "2010-01-01T23:20Z";
         String endDate = "2010-01-02T01:40Z";
         b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            b.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -248,7 +248,7 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
         bundles[0].submitFeedsScheduleProcess(prism);
         InstancesResult r =
                 prism.getProcessHelper().getProcessInstanceResume(
-                        Util.readEntityName(bundles[0].getProcessData()), null);
+                    Util.readEntityName(bundles[0].getProcessData()), null);
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.UNPARSEABLE_DATE);
     }
 
@@ -362,15 +362,8 @@ public class ProcessInstanceResumeTest extends BaseTestClass {
         InstanceUtil.validateResponse(result, 6, 6, 0, 0, 0);
     }
 
-
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-
-        Bundle b = BundleUtil.readELBundle();
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
index 7e39ea2..6bb466c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
@@ -43,6 +43,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -76,10 +77,9 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
         String endDate = "2010-01-02T01:11Z";
 
         bundle.setInputFeedDataPath(feedInputPath);
-        String prefix = bundle.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            bundle.getFeedDataPathPrefix(), dataDates);
     }
 
 
@@ -206,14 +206,8 @@ public class ProcessInstanceRunningTest extends BaseTestClass {
         InstanceUtil.validateSuccessWOInstances(result);
     }
 
-
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle b = BundleUtil.readELBundle();
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
index 1c4f153..5e56037 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
@@ -40,6 +40,7 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction.Status;
 import org.apache.oozie.client.Job;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -434,4 +435,9 @@ public class ProcessInstanceStatusTest extends BaseTestClass {
                 "?start=2010-01-02T01:00Z&end=2010-01-02T01:11Z");
         InstanceUtil.validateFailedInstances(r, 3);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
index cbf5ef9..02bd254 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
@@ -45,6 +45,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -75,11 +76,10 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
         String startDate = "2010-01-01T23:40Z";
         String endDate = "2010-01-02T01:40Z";
         bundle.setInputFeedDataPath(feedInputPath);
-        String prefix = bundle.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
 
         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            bundle.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -314,12 +314,7 @@ public class ProcessInstanceSuspendTest extends BaseTestClass {
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle bundle = BundleUtil.readELBundle();
-        bundle = new Bundle(bundle, cluster);
-        bundle.setInputFeedDataPath(feedInputPath);
-        String prefix = bundle.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
index 7647d15..fa02fdf 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
@@ -33,11 +33,13 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job.Status;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -133,4 +135,9 @@ public class ProcessLibPathTest extends BaseTestClass {
         OozieUtil.createMissingDependencies(cluster, EntityType.PROCESS, processName, 0);
         InstanceUtil.waitForBundleToReachState(cluster, processName, Status.SUCCEEDED);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
index d994780..5d1f1b6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
@@ -50,6 +50,7 @@ import org.testng.annotations.AfterMethod;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.List;
@@ -109,10 +110,11 @@ public class HCatFeedOperationsTest extends BaseTestClass {
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteTable() throws HCatException {
+    public void tearDownClass() throws IOException {
         clusterHC.dropTable(dbName, tableName, true);
         clusterHC.dropTable(dbName, randomTblName, true);
         cluster2HC.dropTable(dbName, tableName, true);
+        cleanTestDirs();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatProcessTest.java
index 32062ba..980b96f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatProcessTest.java
@@ -51,11 +51,13 @@ import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -646,4 +648,9 @@ public class HCatProcessTest extends BaseTestClass {
     public void tearDown() {
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
index 5452f50..557a9f1 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
@@ -48,6 +48,7 @@ import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
 import org.joda.time.format.DateTimeFormat;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -356,4 +357,9 @@ public class HCatReplicationTest extends BaseTestClass {
     public void tearDown() {
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
index cce2860..5dc96e2 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
@@ -46,6 +46,7 @@ import org.apache.oozie.client.OozieClient;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -287,4 +288,8 @@ public class HCatRetentionTest extends BaseTestClass {
         return MathUtil.crossProduct(periods, retentionUnits, dataTypes);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
index c65526a..838c6d7 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
@@ -39,11 +39,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.List;
@@ -224,4 +226,8 @@ public class LineageApiProcessInstanceTest extends BaseTestClass {
 
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
index f7f95da..37aa98c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
@@ -43,11 +43,13 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.http.HttpResponse;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -606,7 +608,8 @@ public class LineageApiTest extends BaseTestClass {
 
     @Test
     public void testColoToClusterNode() throws Exception {
-        final VerticesResult verticesResult = lineageHelper.getVerticesByType(Vertex.VERTEX_TYPE.COLO);
+        final VerticesResult verticesResult = lineageHelper.getVerticesByType(
+            Vertex.VERTEX_TYPE.COLO);
         GraphAssert.assertVertexSanity(verticesResult);
         Assert.assertTrue(verticesResult.getTotalSize() > 0, "Expected at least 1 colo node");
         Assert.assertTrue(verticesResult.getTotalSize() <= 3, "Expected at most 3 colo nodes");
@@ -653,4 +656,9 @@ public class LineageApiTest extends BaseTestClass {
             AssertUtil.checkForListSize(userIncoming.filterByName(feed.getName()), 1);
         }
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
index 9bed6f2..5000746 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
@@ -33,12 +33,14 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 import javax.xml.bind.JAXBException;
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 /*
@@ -101,14 +103,17 @@ public class EntityDryRunTest extends BaseTestClass {
      */
     @Test(groups = {"singleCluster"})
     public void testDryRunFailureUpdateProcess() throws Exception {
-        bundles[0].setProcessValidity(TimeUtil.getTimeWrtSystemTime(-10), TimeUtil.getTimeWrtSystemTime(100));
+        bundles[0].setProcessValidity(TimeUtil.getTimeWrtSystemTime(-10),
+            TimeUtil.getTimeWrtSystemTime(100));
         bundles[0].submitAndScheduleProcess();
         bundles[0].setProcessProperty("EntityDryRunTestProp", "${coord:someEL(1)");
         ServiceResponse response = prism.getProcessHelper().update(bundles[0].getProcessData(),
                 bundles[0].getProcessData(), TimeUtil.getTimeWrtSystemTime(5), null);
         validate(response);
-        Assert.assertEquals(OozieUtil.getNumberOfBundle(cluster, EntityType.PROCESS, bundles[0].getProcessName()), 1,
-                "more than one bundle found after failed update request");
+        Assert.assertEquals(
+            OozieUtil.getNumberOfBundle(cluster, EntityType.PROCESS, bundles[0].getProcessName()),
+            1,
+            "more than one bundle found after failed update request");
     }
 
     /**
@@ -120,7 +125,8 @@ public class EntityDryRunTest extends BaseTestClass {
         String feed = bundles[0].getInputFeedFromBundle();
         feed = Util.setFeedProperty(feed, "EntityDryRunTestProp", "${coord:someEL(1)");
         bundles[0].submitClusters(prism);
-        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(
+            Util.URLS.SUBMIT_AND_SCHEDULE_URL, feed);
         validate(response);
     }
 
@@ -137,14 +143,22 @@ public class EntityDryRunTest extends BaseTestClass {
         feed = Util.setFeedProperty(feed, "EntityDryRunTestProp", "${coord:someEL(1)");
         response = prism.getFeedHelper().update(feed, feed);
         validate(response);
-        Assert.assertEquals(OozieUtil.getNumberOfBundle(cluster, EntityType.FEED, Util.readEntityName(feed)), 1,
-                "more than one bundle found after failed update request");
+        Assert.assertEquals(
+            OozieUtil.getNumberOfBundle(cluster, EntityType.FEED, Util.readEntityName(feed)), 1,
+            "more than one bundle found after failed update request");
     }
 
     private void validate(ServiceResponse response) throws JAXBException {
         AssertUtil.assertFailed(response);
-        Assert.assertTrue(response.getMessage().contains("org.apache.falcon.FalconException: AUTHENTICATION : E1004 :" +
-                        " E1004: Expression language evaluation error, Unable to evaluate :${coord:someEL(1)"),
-                "Correct response was not present in process / feed schedule");
+        Assert.assertTrue(response.getMessage()
+            .contains("org.apache.falcon.FalconException: AUTHENTICATION : E1004 :" +
+                " E1004: Expression language evaluation error, Unable to evaluate :${coord:someEL" +
+                "(1)"),
+            "Correct response was not present in process / feed schedule");
+    }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
index f0d5a50..564637c 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
@@ -30,11 +30,13 @@ import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 @Test(groups = "distributed")
@@ -118,4 +120,9 @@ public class FeedDelayParallelTimeoutTest extends BaseTestClass {
         prism.getFeedHelper()
             .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feedOutput01);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
index f20877f..666a62f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
@@ -34,11 +34,13 @@ import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -205,4 +207,9 @@ public class FeedRetentionTest extends BaseTestClass {
         AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOutput01));
         AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feedOutput02));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
index f4553df..884862b 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
@@ -52,6 +52,7 @@ import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Minutes;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -1706,4 +1707,8 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         return null;
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
index 01c00a9..0c547ab 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
@@ -33,11 +33,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -333,4 +335,9 @@ public class OptionalInputTest extends BaseTestClass {
         InstanceUtil.waitTillInstanceReachState(oozieClient, processName,
                 2, CoordinatorAction.Status.KILLED, EntityType.PROCESS);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
index 24af96e..a4c2dae 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
@@ -28,11 +28,13 @@ import org.apache.falcon.regression.core.util.Util;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.Arrays;
 import java.util.List;
@@ -431,4 +433,8 @@ public class PrismClusterDeleteTest extends BaseTestClass {
             store1.toArray(new String[store1.size()])), "DataStores are not equal!");
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
index 1ef97ed..630ca28 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
@@ -32,11 +32,13 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 
@@ -267,4 +269,8 @@ public class PrismConcurrentRequestTest extends BaseTestClass {
         }
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
index f4dfe78..b1e78d0 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
@@ -37,11 +37,13 @@ import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -1125,4 +1127,8 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         return temp;
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
index 77d07af..71abd95 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
@@ -36,11 +36,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.WorkflowJob;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -670,4 +672,9 @@ public class PrismFeedLateReplicationTest extends BaseTestClass {
                 0) == 1,
             "id have to be equal 1");
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
index 83759b1..ee7052e 100755
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
@@ -40,6 +40,7 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -895,4 +896,9 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
         AssertUtil.assertFailed(r, "is defined more than once for feed");
         Assert.assertTrue(r.getMessage().contains("is defined more than once for feed"));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
index af4243b..6aebbd7 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction.Status;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -308,4 +309,9 @@ public class PrismFeedReplicationUpdateTest extends BaseTestClass {
         logger.info("updated feed: " + Util.prettyPrintXml(feed01));
         AssertUtil.assertSucceeded(prism.getFeedHelper().update(feed01, feed01));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
index 4c82baf..af1620a 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
@@ -30,11 +30,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 @Test(groups = "distributed")
@@ -351,4 +353,8 @@ public class PrismFeedResumeTest extends BaseTestClass {
                 coloHelper.getFeedHelper().getStatus(Util.URLS.STATUS_URL, entity)).getMessage());
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
index dc82c39..ab2e95d 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
@@ -29,6 +29,7 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -89,4 +90,9 @@ public class PrismFeedScheduleTest extends BaseTestClass {
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.checkNotStatus(cluster1OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
index e9a9cf4..b7da224 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
@@ -38,11 +38,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 public class PrismFeedSnSTest extends BaseTestClass {
@@ -445,4 +447,8 @@ public class PrismFeedSnSTest extends BaseTestClass {
         AssertUtil.checkNotStatus(cluster1OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
index d6dbaa5..843990f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
@@ -30,11 +30,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.TestNGException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 public class PrismFeedSuspendTest extends BaseTestClass {
@@ -358,4 +360,8 @@ public class PrismFeedSuspendTest extends BaseTestClass {
         }
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
index b6bf6d6..89f3686 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
@@ -40,11 +40,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.List;
@@ -267,4 +269,8 @@ public class PrismFeedUpdateTest extends BaseTestClass {
             bundles[0].getProcessName()),2);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
index 1fe768a..18c29f3 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
@@ -31,11 +31,13 @@ import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.TestNGException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -1005,5 +1007,8 @@ public class PrismProcessDeleteTest extends BaseTestClass {
         return temp;
     }
 
-
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
index 2e0fbaf..4ac57fc 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
@@ -30,11 +30,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.TestNGException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 @Test(groups = "distributed")
@@ -507,4 +509,9 @@ public class PrismProcessResumeTest extends BaseTestClass {
         }
 
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }


[41/41] git commit: Merge remote-tracking branch master into FALCON-585.

Posted by ra...@apache.org.
Merge remote-tracking branch master into FALCON-585.


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

Branch: refs/heads/FALCON-585
Commit: 0d3e97d784a9b33367ca67af99935f3d8aa3c4d0
Parents: 1f9a27f 5766b74
Author: Raghav Kumar Gautam <ra...@apache.org>
Authored: Tue Sep 9 20:12:46 2014 -0700
Committer: Raghav Kumar Gautam <ra...@apache.org>
Committed: Wed Sep 10 14:00:25 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  42 ++
 .../java/org/apache/falcon/cli/FalconCLI.java   |  89 +++-
 .../org/apache/falcon/client/FalconClient.java  | 177 +++++--
 .../org/apache/falcon/resource/EntityList.java  |   2 +-
 .../falcon/resource/EntitySummaryResult.java    | 220 +++++++++
 .../org/apache/falcon/entity/EntityUtil.java    |  68 +++
 .../InstanceRelationshipGraphBuilder.java       | 107 ++++-
 .../falcon/metadata/MetadataMappingService.java |   8 +-
 .../metadata/RelationshipGraphBuilder.java      |  13 +-
 .../falcon/metadata/RelationshipLabel.java      |   8 +-
 .../apache/falcon/retention/EvictionHelper.java |  88 ++++
 .../falcon/workflow/WorkflowExecutionArgs.java  |   8 +-
 .../workflow/WorkflowExecutionContext.java      |  31 +-
 .../WorkflowJobEndNotificationService.java      |   5 +
 .../apache/falcon/entity/EntityUtilTest.java    |  33 ++
 .../metadata/MetadataMappingServiceTest.java    | 421 +++++++++++++----
 .../workflow/WorkflowExecutionContextTest.java  |   4 +-
 .../WorkflowJobEndNotificationServiceTest.java  |   4 +-
 docs/src/site/twiki/FalconCLI.twiki             |  64 ++-
 docs/src/site/twiki/restapi/EntityList.twiki    |  38 +-
 docs/src/site/twiki/restapi/EntitySummary.twiki |  73 +++
 docs/src/site/twiki/restapi/InstanceList.twiki  |  29 +-
 docs/src/site/twiki/restapi/InstanceLogs.twiki  |  32 +-
 .../site/twiki/restapi/InstanceRunning.twiki    |  26 +-
 .../src/site/twiki/restapi/InstanceStatus.twiki |  30 +-
 .../site/twiki/restapi/InstanceSummary.twiki    |  17 +-
 docs/src/site/twiki/restapi/ResourceList.twiki  |  13 +-
 falcon-regression/CHANGES.txt                   |  55 ++-
 .../falcon/regression/core/bundle/Bundle.java   |  54 +--
 .../core/enumsAndConstants/MerlinConstants.java |   4 +
 .../core/helpers/ClusterEntityHelperImpl.java   |  14 +-
 .../core/interfaces/IEntityManagerHelper.java   | 130 +++--
 .../regression/core/supportClasses/Brother.java |  16 +-
 .../falcon/regression/core/util/BundleUtil.java |   9 +-
 .../regression/core/util/CleanupUtil.java       |  11 +-
 .../falcon/regression/core/util/HadoopUtil.java |  66 +--
 .../regression/core/util/KerberosHelper.java    |   3 +-
 .../falcon/regression/core/util/Util.java       |  85 ++--
 .../regression/testHelper/BaseTestClass.java    |   9 +
 .../falcon/regression/AuthorizationTest.java    | 127 ++---
 .../regression/ELExp_FutureAndLatestTest.java   |  20 +-
 .../falcon/regression/ELValidationsTest.java    |   5 +
 .../regression/EmbeddedPigScriptTest.java       |  83 ++--
 .../regression/FeedClusterUpdateTest.java       |  83 ++--
 .../regression/FeedInstanceStatusTest.java      | 188 +++-----
 .../falcon/regression/FeedReplicationTest.java  |  23 +-
 .../falcon/regression/FeedResumeTest.java       |  36 +-
 .../falcon/regression/FeedScheduleTest.java     |  41 +-
 .../falcon/regression/FeedStatusTest.java       |  41 +-
 .../regression/FeedSubmitAndScheduleTest.java   |  57 +--
 .../falcon/regression/FeedSubmitTest.java       |  35 +-
 .../falcon/regression/FeedSuspendTest.java      |  42 +-
 .../falcon/regression/InstanceParamTest.java    |  52 +-
 .../falcon/regression/InstanceSummaryTest.java  | 182 +++----
 .../apache/falcon/regression/NewRetryTest.java  | 103 ++--
 .../falcon/regression/NoOutputProcessTest.java  |  28 +-
 .../ProcessInstanceColoMixedTest.java           |  21 +-
 .../regression/ProcessInstanceKillsTest.java    | 118 ++---
 .../regression/ProcessInstanceRerunTest.java    | 159 +++----
 .../regression/ProcessInstanceResumeTest.java   | 225 +++------
 .../regression/ProcessInstanceRunningTest.java  |  70 +--
 .../regression/ProcessInstanceStatusTest.java   | 160 +++----
 .../regression/ProcessInstanceSuspendTest.java  | 129 ++---
 .../falcon/regression/ProcessLibPathTest.java   |  46 +-
 .../regression/hcat/HCatFeedOperationsTest.java | 269 +++++++++++
 .../falcon/regression/hcat/HCatProcessTest.java |   7 +
 .../regression/hcat/HCatReplicationTest.java    |  16 +-
 .../regression/hcat/HCatRetentionTest.java      |  13 +-
 .../lineage/LineageApiProcessInstanceTest.java  |  11 +-
 .../regression/lineage/LineageApiTest.java      |  25 +-
 .../regression/prism/EntityDryRunTest.java      |  45 +-
 .../prism/FeedDelayParallelTimeoutTest.java     |  13 +-
 .../regression/prism/FeedRetentionTest.java     | 219 ---------
 .../prism/NewPrismProcessUpdateTest.java        | 121 ++---
 .../regression/prism/OptionalInputTest.java     | 150 +++---
 .../prism/PrismClusterDeleteTest.java           |  34 +-
 .../prism/PrismConcurrentRequestTest.java       |  32 +-
 .../regression/prism/PrismFeedDeleteTest.java   | 133 ++----
 .../prism/PrismFeedLateReplicationTest.java     |  19 +-
 .../PrismFeedReplicationPartitionExpTest.java   |  79 ++--
 .../prism/PrismFeedReplicationUpdateTest.java   |  33 +-
 .../regression/prism/PrismFeedResumeTest.java   | 142 +++---
 .../regression/prism/PrismFeedScheduleTest.java |  10 +-
 .../regression/prism/PrismFeedSnSTest.java      | 288 ++++++------
 .../regression/prism/PrismFeedSuspendTest.java  | 113 ++---
 .../regression/prism/PrismFeedUpdateTest.java   |  94 ++--
 .../prism/PrismProcessDeleteTest.java           | 101 ++--
 .../prism/PrismProcessResumeTest.java           | 166 +++----
 .../prism/PrismProcessScheduleTest.java         | 167 +++----
 .../regression/prism/PrismProcessSnSTest.java   | 164 +++----
 .../prism/PrismProcessSuspendTest.java          | 104 ++--
 .../regression/prism/PrismSubmitTest.java       | 114 ++---
 .../prism/ProcessPartitionExpVariableTest.java  |   7 +-
 .../prism/RescheduleKilledProcessTest.java      |  39 +-
 .../RescheduleProcessInFinalStatesTest.java     |  31 +-
 .../falcon/regression/prism/RetentionTest.java  |  87 +++-
 .../prism/UpdateAtSpecificTimeTest.java         | 469 +++++++------------
 .../falcon/regression/ui/LineageGraphTest.java  |   8 +-
 .../falcon/regression/ui/ProcessUITest.java     |  36 +-
 .../FETL-BillingRC.xml                          |  14 +-
 .../test/resources/impressionRC/cluster-0.1.xml |  43 --
 .../impressionRC/feed-FETL-ImpressionRC.xml     |  45 --
 .../impressionRC/feed-FETL-RequestRC.xml        |  46 --
 .../resources/impressionRC/feed-FETL2-RRLog.xml |  33 --
 .../process-FETL-ImpressionRC-Conversion.xml    |  45 --
 falcon-regression/pom.xml                       |   4 -
 .../falcon/messaging/JMSMessageConsumer.java    |  36 +-
 .../falcon/messaging/JMSMessageProducer.java    |  20 +-
 .../falcon/messaging/FeedProducerTest.java      |  12 +-
 .../messaging/JMSMessageConsumerTest.java       |   4 +-
 .../messaging/JMSMessageProducerTest.java       |  41 +-
 .../falcon/messaging/ProcessProducerTest.java   |  12 +-
 .../org/apache/falcon/aspect/GenericAlert.java  |   7 +
 .../feed/FeedReplicationCoordinatorBuilder.java |  15 +-
 .../feed/FeedRetentionCoordinatorBuilder.java   |   4 +-
 .../ProcessExecutionCoordinatorBuilder.java     |  26 +-
 .../falcon/workflow/FalconPostProcessing.java   |   7 +-
 .../src/main/resources/action/post-process.xml  |   2 +-
 oozie/src/main/resources/action/pre-process.xml |   4 +-
 .../feed/OozieFeedWorkflowBuilderTest.java      |   9 +-
 .../falcon/oozie/process/AbstractTestBase.java  |  13 +-
 .../OozieProcessWorkflowBuilderTest.java        |  16 +-
 .../workflow/FalconPostProcessingTest.java      |  56 ++-
 pom.xml                                         |   6 +
 .../falcon/resource/AbstractEntityManager.java  | 180 ++++---
 .../resource/AbstractInstanceManager.java       | 234 ++++-----
 .../AbstractSchedulableEntityManager.java       | 107 ++++-
 .../resource/proxy/InstanceManagerProxy.java    |  22 +-
 .../proxy/SchedulableEntityManagerProxy.java    |  55 ++-
 .../falcon/resource/EntityManagerTest.java      |  66 ++-
 .../metadata/LineageMetadataResourceTest.java   |   4 +-
 .../apache/falcon/latedata/LateDataHandler.java |  10 +-
 .../falcon/rerun/handler/LateRerunConsumer.java |  22 +-
 .../rerun/handler/TestLateRerunHandler.java     |  13 +
 .../apache/falcon/retention/FeedEvictor.java    |  30 +-
 .../falcon/cluster/util/EmbeddedCluster.java    |  24 +-
 .../apache/falcon/resource/InstanceManager.java |  21 +-
 .../resource/SchedulableEntityManager.java      |  30 +-
 .../java/org/apache/falcon/cli/FalconCLIIT.java | 107 ++++-
 .../apache/falcon/late/LateDataHandlerIT.java   |   4 +-
 .../falcon/resource/EntityManagerJerseyIT.java  |  18 +-
 .../org/apache/falcon/resource/TestContext.java |   5 +-
 142 files changed, 4842 insertions(+), 4169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --cc falcon-regression/CHANGES.txt
index 2e2b447,b74a4ee..2f5de05
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@@ -5,25 -5,44 +5,62 @@@ Trunk (Unreleased
    INCOMPATIBLE CHANGES
  
    NEW FEATURES
+    FALCON-589 Add test cases for various feed operations on Hcat feeds (Karishma G 
+    via Samarth Gupta)
  
    IMPROVEMENTS
-    
-    FALCON-629 Add javadoc for method stitchTwoHdfsPaths (Raghav Kumar Gautam
-    via Samarth Gupta) 
- 
 +   FALCON-585 Remove requirement to have write access to / (Raghav Kumar Gautam)
++      FALCON-680: Tests using LocalDC, impressionRC, updateBundle should stop using root dir
++      (Raghav Kumar Gautam)
++
++      FALCON-679: Tests using ELbundle should stop writing to root dir (Raghav Kumar Gautam) 
++
 +      FALCON-641 Tests using LateDataBundles should stop using root dir (Raghav 
 +      Kumar Gautam via Arpit Gupta)
++
 +      FALCON-633 RetryTests and Retentions tests should stop using root dir
 +      (Raghav Kumar Gautam via Samarth Gupta)
++
++      FALCON-629 Add javadoc for method stitchTwoHdfsPaths (Raghav Kumar Gautam
++      via Samarth Gupta) 
++
 +      FALCON-606 hcat tests should stop using root dir (Raghav Kumar Gautam 
 +      via Arpit Gupta)
 +
-    FALCON-619 ELExp_FutureAndLatestTest stabilization (Paul Isaychuk via 
-    Arpit Gupta)
+    FALCON-675 Request URLS moved from parameters into methods in falcon-regression (Ruslan
+    Ostafiychuk)
+ 
+    FALCON-656 add test in falcon regression's Authorization test where non-feed owner updates
+    a feed with a dependent process(Karishma via Samarth Gupta)
+ 
+    FALCON-674 General code factored out for ProcessInstance* tests (Paul Isaychuk via Ruslan
+    Ostafiychuk)
+ 
+    FALCON-657 String datePattern moved to BaseTestClass (Ruslan Ostafiychuk)
  
-    FALCON-610 Refactoring and documentation updates (Paul Isaychuk via 
-    Arpit Gupta)
+    FALCON-643 Tests with zero-output/input scenario amended to match test case (Paul Isaychuk via
+    Ruslan Ostafiychuk)
+ 
+    FALCON-660 7 test classes refactored and few of them documented (Paul Isaychuk via
+    Ruslan Ostafiychuk)
+ 
+    FALCON-653 Add falcon regression test for zero input process(Karishma via Samarth Gupta)
+    FALCON-655 Skip workflow upload if process won't be submitted (Ruslan Ostafiychuk)
+ 
+    FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if
+    clean_test_dir=false (Ruslan Ostafiychuk)
+ 
+    FALCON-646 Refactoring, documentation stuff (Paul Isaychuk via Ruslan Ostafiychuk)
+ 
+    FALCON-572 HadoopUtil cleanup in falcon-regression (Ruslan Ostafiychuk via Samarth Gupta)
+    FALCON-632 Refactoring, documentation stuff (Paul Isaychuk via Samarth Gupta)
+ 
+    FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored
+    (Paul Isaychuk via Samarth Gupta)
+ 
+    FALCON-619 ELExp_FutureAndLatestTest stabilization (Paul Isaychuk via Arpit Gupta)
 -
++   
+    FALCON-610 Refactoring and documentation updates (Paul Isaychuk via Arpit Gupta)
  
     FALCON-581 Refactor code for cross product and make it a method 
     (Raghav Kumar Gautam via Arpit Gupta) 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
index ff5a6a8,6d8b032..1536d2c
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/bundle/Bundle.java
@@@ -44,10 -44,8 +44,9 @@@ import org.apache.falcon.regression.Ent
  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.HadoopUtil;
  import org.apache.falcon.regression.core.util.TimeUtil;
  import org.apache.falcon.regression.core.util.Util;
- import org.apache.falcon.regression.core.util.Util.URLS;
  import org.apache.hadoop.security.authentication.client.AuthenticationException;
  import org.apache.log4j.Logger;
  import org.joda.time.DateTime;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
index 166fb3d,922c030..62838f8
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/BundleUtil.java
@@@ -44,56 -44,38 +44,52 @@@ public final class BundleUtil 
      }
      private static final Logger LOGGER = Logger.getLogger(BundleUtil.class);
  
 -    public static Bundle readLateDataBundle() throws IOException {
 -        return readBundleFromFolder("LateDataBundles");
 +    public static Bundle readLateDataBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("LateDataBundles", appPath, testName);
      }
  
 -    public static Bundle readRetryBundle() throws IOException {
 -        return readBundleFromFolder("RetryTests");
 +    public static Bundle readRetryBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("RetryTests", appPath, testName);
      }
  
 -    public static Bundle readRetentionBundle() throws IOException {
 -        return readBundleFromFolder("RetentionBundles");
 +    public static Bundle readRetentionBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("RetentionBundles", appPath, testName);
      }
  
 -    public static Bundle readELBundle() throws IOException {
 -        return readBundleFromFolder("ELbundle");
 +    public static Bundle readELBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("ELbundle", appPath, testName);
      }
  
 -    public static Bundle readHCatBundle() throws IOException {
 -        return readBundleFromFolder("hcat");
 +    public static Bundle readHCatBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("hcat", appPath, testName);
      }
  
 -    public static Bundle readHCat2Bundle() throws IOException {
 -        return readBundleFromFolder("hcat_2");
 +    public static Bundle readHCat2Bundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("hcat_2", appPath, testName);
      }
  
 -    public static Bundle readLocalDCBundle() throws IOException {
 -        return readBundleFromFolder("LocalDC_feedReplicaltion_BillingRC");
 +    public static Bundle readLocalDCBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("LocalDC_feedReplicaltion_BillingRC", appPath, testName);
      }
  
-     public static Bundle readImpressionRCBundle(String appPath, String testName) throws IOException {
-         return generateBundleFromTemplate("impressionRC", appPath, testName);
-     }
- 
 -    public static Bundle readUpdateBundle() throws IOException {
 -        return readBundleFromFolder("updateBundle");
 +    public static Bundle readUpdateBundle(String appPath, String testName) throws IOException {
 +        return generateBundleFromTemplate("updateBundle", appPath, testName);
      }
  
 +    /** Generate a bundle from the template stored in a directory
 +     * @param templatePath name of directory where template is stored
 +     * @param appPath application path where staging/temp/working directory will live
 +     * @param testName name of test
 +     * @return the customized bundle
 +     * @throws IOException
 +     */
 +    private static Bundle generateBundleFromTemplate(String templatePath, String appPath,
 +            String testName) throws IOException {
 +        final Bundle bundle = readBundleFromFolder(templatePath);
 +        bundle.updateClusterLocations(HadoopUtil.stitchHdfsPath(appPath, testName));
 +        return bundle;
 +    }
- 
++    
      private static Bundle readBundleFromFolder(final String folderPath) throws IOException {
          LOGGER.info("Loading xmls from directory: " + folderPath);
          File directory = null;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
index c348348,af5c01a..735a732
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
@@@ -394,60 -368,4 +368,32 @@@ public final class HadoopUtil 
          copyDataToFolders(fs, folderPrefix, folderPaths,
              OSUtil.NORMAL_INPUT + "_SUCCESS", OSUtil.NORMAL_INPUT + "log_01.txt");
      }
 +
 +    /**
-      * Removes general folder on file system. Creates folders according to passed folders list
-      * and fills them with data if required.
-      *
-      * @param fileSystem destination file system
-      * @param prefix prefix of path where data should be placed
-      * @param folderList list of folders to be created and filled with data if required
-      * @param uploadData if folders should be filled with data
-      * @throws IOException
-      */
-     public static void replenishData(FileSystem fileSystem, String prefix, List<String> folderList,
-         boolean uploadData) throws IOException {
-         //purge data first
-         deleteDirIfExists(prefix, fileSystem);
- 
-         folderList.add(SOMETHING_RANDOM);
- 
-         for (final String folder : folderList) {
-             final String pathString = prefix + folder;
-             LOGGER.info(fileSystem.getUri() + pathString);
-             fileSystem.mkdirs(new Path(pathString));
-             if (uploadData) {
-                 fileSystem.copyFromLocalFile(new Path(OSUtil.RESOURCES + "log_01.txt"),
-                         new Path(pathString));
-             }
-         }
-     }
- 
-     /**
 +     * Stitch two or more hadoop paths. For eg: stitchHdfsPath("/tmp/", "/test") = "/tmp/test",
 +     * stitchHdfsPath("/tmp/", "test") = "/tmp/test", stitchHdfsPath("/tmp", "test") = "/tmp/test"
 +     * @param path1 the first path to be stitched
 +     * @param path2 the second path to be stitched
 +     * @param pathParts other paths to be stitched
 +     * @return final stitched path
 +     */
 +    public static String stitchHdfsPath(String path1, String path2, String... pathParts) {
 +        String retValue = stitchTwoHdfsPaths(path1, path2);
 +        for (String pathPart : pathParts) {
 +            retValue = stitchTwoHdfsPaths(retValue, pathPart);
 +        }
 +        return retValue;
 +    }
 +
 +    /**
 +     * Stitch two hadoop paths. For eg: stitchTwoHdfsPaths("/tmp/", "/test") = "/tmp/test",
 +     * stitchTwoHdfsPaths("/tmp/", "test") = "/tmp/test",
 +     * stitchTwoHdfsPaths("/tmp", "test") = "/tmp/test"
 +     * @param path1 the first path to be stitched
 +     * @param path2 the second path to be stitched
 +     * @return final stitched path
 +     */
 +    private static String stitchTwoHdfsPaths(String path1, String path2) {
 +        return path1.replaceAll("/*$", "") + "/" + path2.replaceAll("^/*", "");
 +    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
index d892332,c16c8f4..bc78699
--- a/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
+++ b/falcon-regression/merlin/src/main/java/org/apache/falcon/regression/testHelper/BaseTestClass.java
@@@ -45,16 -45,14 +45,17 @@@ public class BaseTestClass 
          }
      }
  
 -    public ColoHelper prism;
 -    public List<ColoHelper> servers;
 -    public List<FileSystem> serverFS;
 -    public List<OozieClient> serverOC;
 -    public String baseHDFSDir = "/tmp/falcon-regression";
 +    public final ColoHelper prism;
 +    public final List<ColoHelper> servers;
 +    public final List<FileSystem> serverFS;
 +    public final List<OozieClient> serverOC;
 +    public final String baseHDFSDir = Config.getProperty("falcon.regression.test.dir",
 +        "/tmp/falcon-regression");
 +    public final String baseAppHDFSDir = Config.getProperty("falcon.regression.app.dir",
 +        "/tmp/falcon-regression-app");
      public static final String PRISM_PREFIX = "prism";
      protected Bundle[] bundles;
+     public static final String MINUTE_DATE_PATTERN = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
  
  
      public BaseTestClass() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/AuthorizationTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
index d19daed,25e2dfe..24e94ea
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELExp_FutureAndLatestTest.java
@@@ -50,10 -51,7 +51,9 @@@ public class ELExp_FutureAndLatestTest 
      ColoHelper cluster = servers.get(0);
      FileSystem clusterFS = serverFS.get(0);
      OozieClient clusterOC = serverOC.get(0);
-     private String prefix;
      private String baseTestDir = baseHDFSDir + "/ELExp_FutureAndLatest";
 +    private String testInputDir = baseTestDir + "/input";
 +    private String testOutputDir = baseTestDir + "/output";
      private String aggregateWorkflowDir = baseTestDir + "/aggregator";
      private static final Logger logger = Logger.getLogger(ELExp_FutureAndLatestTest.class);
  
@@@ -69,10 -67,8 +69,8 @@@
          String startDate = TimeUtil.getTimeWrtSystemTime(-20);
          String endDate = TimeUtil.getTimeWrtSystemTime(70);
  
-         b.setInputFeedDataPath(testInputDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
 -        b.setInputFeedDataPath(baseTestDir + "/ELExp_latest/testData" + MINUTE_DATE_PATTERN);
++        b.setInputFeedDataPath(testInputDir + MINUTE_DATE_PATTERN);
          b.setProcessWorkflow(aggregateWorkflowDir);
-         prefix = b.getFeedDataPathPrefix();
-         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
  
          List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 1);
  
@@@ -82,13 -79,11 +81,11 @@@
      @BeforeMethod(alwaysRun = true)
      public void setUp(Method method) throws Exception {
          logger.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0] = new Bundle(bundles[0], cluster);
          bundles[0].generateUniqueBundle();
-         bundles[0].setInputFeedDataPath(
-             testInputDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
-         bundles[0].setOutputFeedLocationData(
-             testOutputDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
 -        bundles[0].setInputFeedDataPath(baseTestDir + "/ELExp_latest/testData" +
 -            MINUTE_DATE_PATTERN);
++        bundles[0].setInputFeedDataPath(testInputDir + MINUTE_DATE_PATTERN);
++        bundles[0].setOutputFeedLocationData(testOutputDir + MINUTE_DATE_PATTERN);
          bundles[0].setInputFeedPeriodicity(5, TimeUnit.minutes);
          bundles[0].setInputFeedValidity("2010-04-01T00:00Z", "2015-04-01T00:00Z");
          String processStart = TimeUtil.getTimeWrtSystemTime(-3);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ELValidationsTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
index 6a9edd3,99ffe37..85fed2f
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/EmbeddedPigScriptTest.java
@@@ -72,26 -73,19 +73,19 @@@ public class EmbeddedPigScriptTest exte
  
      @BeforeClass(alwaysRun = true)
      public void createTestData() throws Exception {
- 
          logger.info("in @BeforeClass");
+ 
          //copy pig script
          HadoopUtil.uploadDir(clusterFS, pigScriptDir, OSUtil.RESOURCES + "pig");
- 
 -        Bundle bundle = BundleUtil.readELBundle();
 +        Bundle bundle = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundle.generateUniqueBundle();
          bundle = new Bundle(bundle, cluster);
- 
          String startDate = "2010-01-02T00:40Z";
          String endDate = "2010-01-02T01:10Z";
- 
          bundle.setInputFeedDataPath(inputPath);
-         prefix = bundle.getFeedDataPathPrefix();
-         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
- 
-         List<String> dataDates =
-             TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
- 
-         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+         List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
+         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+             bundle.getFeedDataPathPrefix(), dataDates);
      }
  
      @BeforeMethod(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedClusterUpdateTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedInstanceStatusTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
index 738d0d1,bb629b8..f7a706d
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedResumeTest.java
@@@ -55,10 -54,9 +54,9 @@@ public class FeedResumeTest extends Bas
      @BeforeMethod(alwaysRun = true)
      public void setup(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0].generateUniqueBundle();
          bundles[0] = new Bundle(bundles[0], cluster);
-         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
          bundles[0].submitClusters(prism);
          feed = bundles[0].getInputFeedFromBundle();
      }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
index 7ae5da4,8500d84..8f0ee4e
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedScheduleTest.java
@@@ -51,14 -50,9 +50,9 @@@ public class FeedScheduleTest extends B
      @BeforeMethod(alwaysRun = true)
      public void setUp(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0] = new Bundle(bundles[0], cluster);
          bundles[0].generateUniqueBundle();
-         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
          Bundle.submitCluster(bundles[0]);
          feed = bundles[0].getInputFeedFromBundle();
      }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
index 75def82,6278513..4ee229f
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedStatusTest.java
@@@ -58,10 -54,9 +54,9 @@@ public class FeedStatusTest extends Bas
      @BeforeMethod(alwaysRun = true)
      public void setUp(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0].generateUniqueBundle();
          bundles[0] = new Bundle(bundles[0], cluster);
-         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
  
          //submit the cluster
          ServiceResponse response =

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
index a658102,db82a8c..2c246de
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitAndScheduleTest.java
@@@ -63,10 -57,10 +57,10 @@@ public class FeedSubmitAndScheduleTest 
      @BeforeMethod(alwaysRun = true)
      public void setUp(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0] = new Bundle(bundles[0], cluster);
          bundles[0].generateUniqueBundle();
-         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+         feed = bundles[0].getDataSets().get(0);
      }
  
      @AfterMethod(alwaysRun = true)
@@@ -179,7 -160,9 +160,9 @@@
          AssertUtil.assertSucceeded(response);
          AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
      }
- }
- 
- 
  
+     @AfterClass(alwaysRun = true)
+     public void tearDownClass() throws IOException {
+         cleanTestDirs();
+     }
 -}
++}

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
index a8795db,8d96741..3bd71ab
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSubmitTest.java
@@@ -48,14 -47,9 +47,9 @@@ public class FeedSubmitTest extends Bas
      @BeforeMethod(alwaysRun = true)
      public void setUp(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0].generateUniqueBundle();
          bundles[0] = new Bundle(bundles[0], cluster);
-         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
  
          //submit the cluster
          ServiceResponse response =

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
index d324b64,9676a30..adab5a6
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedSuspendTest.java
@@@ -51,14 -50,9 +50,9 @@@ public class FeedSuspendTest extends Ba
      @BeforeMethod(alwaysRun = true)
      public void setUp(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0].generateUniqueBundle();
          bundles[0] = new Bundle(bundles[0], cluster);
-         bundles[0].setProcessWorkflow(aggregateWorkflowDir);
  
          //submit the cluster
          ServiceResponse response =

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceParamTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index 2e62360,db600ff..6c873c3
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@@ -57,19 -60,8 +60,9 @@@ import java.util.List
  @Test(groups = "embedded")
  public class InstanceSummaryTest extends BaseTestClass {
  
-     //1. process : test summary single cluster few instance some future some past
-     //2. process : test multiple cluster, full past on one cluster,
-     // full future on one cluster, half future / past on third one
- 
-     // 3. feed : same as test 1 for feed
-     // 4. feed : same as test 2 for feed
- 
- 
      String baseTestHDFSDir = baseHDFSDir + "/InstanceSummaryTest";
-     String feedInputPath = baseTestHDFSDir +
-         "/testInputData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-     String feedOutputPath = baseTestHDFSDir +
-         "/testOutputData/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+     String feedInputPath = baseTestHDFSDir + "/testInputData" + MINUTE_DATE_PATTERN;
++    String feedOutputPath = baseTestHDFSDir + "/testOutputData" + MINUTE_DATE_PATTERN;
      String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
      String startTime;
      String endTime;
@@@ -290,23 -255,23 +257,22 @@@
          feedInputPath, 1);*/
  
          //submit and schedule feed
-         prism.getFeedHelper().submitAndSchedule(Util.URLS
-             .SUBMIT_AND_SCHEDULE_URL, feed);
+         prism.getFeedHelper().submitAndSchedule(feed);
  
          InstancesSummaryResult r = prism.getFeedHelper()
-             .getInstanceSummary(Util.readEntityName(feed),
-                 "?start=" + startTime);
- 
-         r = prism.getFeedHelper()
-             .getInstanceSummary(Util.readEntityName(feed),
-                 "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime,
-                     -20));
+             .getInstanceSummary(Util.readEntityName(feed), "?start=" + startTime);
  
+         r = prism.getFeedHelper().getInstanceSummary(Util.readEntityName(feed),
+             "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime, -20));
      }
  
- 
      @AfterMethod(alwaysRun = true)
      public void tearDown() throws IOException {
 -        processBundle.deleteBundle(prism);
 -        removeBundles();
 +        removeBundles(processBundle);
      }
+ 
+     @AfterClass(alwaysRun = true)
+     public void tearDownClass() throws IOException {
+         cleanTestDirs();
+     }
  }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
index 8bcc797,d4f31e9..27d4fdf
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NewRetryTest.java
@@@ -74,11 -75,8 +75,10 @@@ public class NewRetryTest extends BaseT
      DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy/MM/dd/HH/mm");
      final private String baseTestDir = baseHDFSDir + "/NewRetryTest";
      final private String aggregateWorkflowDir = baseTestDir + "/aggregator";
 -    final private String lateDir = baseTestDir + "/lateDataTest/testFolders";
 -    final private String latePath = lateDir + MINUTE_DATE_PATTERN;
 +    final private String lateInputDir = baseTestDir + "/lateDataTest/inputFolders/";
-     final private String lateInputPath = lateInputDir + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
++    final private String lateInputPath = lateInputDir + MINUTE_DATE_PATTERN;
 +    final private String lateOutputDir = baseTestDir + "/lateDataTest/outputFolders/";
-     final private String lateOutputPath = lateOutputDir
-         + "${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
++    final private String lateOutputPath = lateOutputDir + MINUTE_DATE_PATTERN;
      private DateTime startDate;
      private DateTime endDate;
  
@@@ -182,11 -178,11 +182,11 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              //now wait till the process is over
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
  
@@@ -240,11 -236,11 +240,11 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
  
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@@ -293,10 -289,10 +293,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
  
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
@@@ -349,10 -345,10 +349,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@@ -397,10 -393,10 +397,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@@ -449,10 -444,10 +448,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@@ -503,10 -498,10 +502,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@@ -555,10 -550,10 +554,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
@@@ -594,10 -589,10 +593,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
  
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
@@@ -648,10 -643,10 +647,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()),
@@@ -703,10 -698,10 +702,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
              List<DateTime> dates = null;
@@@ -800,13 -795,13 +799,13 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              List<String> initialData =
                  Util.getHadoopDataFromDir(clusterFS, bundles[0].getInputFeedFromBundle(),
 -                    lateDir);
 +                    lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);
              List<DateTime> dates = null;
@@@ -878,10 -873,10 +877,10 @@@
              AssertUtil.assertFailed(response);
          } else {
              AssertUtil.assertSucceeded(response);
 -            HadoopUtil.deleteDirIfExists(lateDir, clusterFS);
 -            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateDir);
 +            HadoopUtil.deleteDirIfExists(lateInputDir, clusterFS);
 +            HadoopUtil.lateDataReplenish(clusterFS, 20, 0, lateInputDir);
              AssertUtil.assertSucceeded(
-                 prism.getProcessHelper().schedule(URLS.SCHEDULE_URL, bundles[0].getProcessData()));
+                 prism.getProcessHelper().schedule(bundles[0].getProcessData()));
              //now wait till the process is over
              String bundleId = OozieUtil.getBundles(clusterOC,
                  Util.readEntityName(bundles[0].getProcessData()), EntityType.PROCESS).get(0);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
index 86faf3b,638768d..cf91d83
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/NoOutputProcessTest.java
@@@ -54,24 -57,19 +57,20 @@@ public class NoOutputProcessTest extend
      FileSystem clusterFS = serverFS.get(0);
      OozieClient clusterOC = serverOC.get(0);
      String testDir = baseHDFSDir + "/NoOutputProcessTest";
-     String inputPath = testDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-     String outputPath = testDir + "/output/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
+     String inputPath = testDir + "/input" + MINUTE_DATE_PATTERN;
++    String outputPath = testDir + "/output/" + MINUTE_DATE_PATTERN;
      String aggregateWorkflowDir = testDir + "/aggregator";
      private static final Logger logger = Logger.getLogger(NoOutputProcessTest.class);
  
      @BeforeClass(alwaysRun = true)
      public void createTestData() throws Exception {
- 
          logger.info("in @BeforeClass");
          HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
- 
 -        Bundle b = BundleUtil.readELBundle();
 +        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          b.generateUniqueBundle();
          b = new Bundle(b, cluster);
- 
          String startDate = "2010-01-03T00:00Z";
          String endDate = "2010-01-03T03:00Z";
- 
          b.setInputFeedDataPath(inputPath);
          String prefix = b.getFeedDataPathPrefix();
          HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
@@@ -89,9 -85,11 +86,12 @@@
          bundles[0] = new Bundle(bundles[0], cluster);
          bundles[0].setProcessWorkflow(aggregateWorkflowDir);
          bundles[0].setInputFeedDataPath(inputPath);
 +        bundles[0].setOutputFeedLocationData(outputPath);
          bundles[0].setProcessValidity("2010-01-03T02:30Z", "2010-01-03T02:45Z");
          bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
+         ProcessMerlin process = new ProcessMerlin(bundles[0].getProcessData());
+         process.setOutputs(null);
+         bundles[0].setProcessData(process.toString());
          bundles[0].submitFeedsScheduleProcess(prism);
      }
  

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
index 03fdd94,7b938ec..68b308e
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceKillsTest.java
@@@ -66,27 -66,21 +66,21 @@@ public class ProcessInstanceKillsTest e
      public void createTestData() throws Exception {
          LOGGER.info("in @BeforeClass");
          HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
- 
 -        Bundle b = BundleUtil.readELBundle();
 +        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          b.generateUniqueBundle();
          b = new Bundle(b, cluster);
- 
          String startDate = "2010-01-01T23:20Z";
          String endDate = "2010-01-02T01:21Z";
- 
          b.setInputFeedDataPath(feedInputPath);
-         String prefix = b.getFeedDataPathPrefix();
-         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
- 
          List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+             b.getFeedDataPathPrefix(), dataDates);
      }
  
      @BeforeMethod(alwaysRun = true)
      public void setup(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());
- 
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0] = new Bundle(bundles[0], cluster);
          bundles[0].generateUniqueBundle();
          bundles[0].setProcessWorkflow(aggregateWorkflowDir);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRerunTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceResumeTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
index 422c6b3,7a7e735..8934eb7
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceRunningTest.java
@@@ -67,22 -66,17 +66,17 @@@ public class ProcessInstanceRunningTes
      public void createTestData() throws Exception {
          LOGGER.info("in @BeforeClass");
          HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
- 
 -        Bundle bundle = BundleUtil.readELBundle();
 +        Bundle bundle = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundle.generateUniqueBundle();
          bundle = new Bundle(bundle, cluster);
- 
          String startDate = "2010-01-02T00:40Z";
          String endDate = "2010-01-02T01:11Z";
- 
          bundle.setInputFeedDataPath(feedInputPath);
-         String prefix = bundle.getFeedDataPathPrefix();
-         HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
          List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
-         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+         HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+             bundle.getFeedDataPathPrefix(), dataDates);
      }
  
- 
      @BeforeMethod(alwaysRun = true)
      public void setup(Method method) throws Exception {
          LOGGER.info("test name: " + method.getName());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
index 4f06501,12a3907..df959b5
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceStatusTest.java
@@@ -75,21 -76,15 +76,16 @@@ public class ProcessInstanceStatusTest 
      @BeforeClass(alwaysRun = true)
      public void createTestData() throws Exception {
          LOGGER.info("in @BeforeClass");
- 
          HadoopUtil.uploadDir(clusterFS, aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
 -        Bundle bundle = BundleUtil.readELBundle();
 +
 +        Bundle bundle = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundle.generateUniqueBundle();
          bundle = new Bundle(bundle, cluster);
- 
          String startDate = "2010-01-01T23:40Z";
          String endDate = "2010-01-02T02:40Z";
- 
          bundle.setInputFeedDataPath(feedInputPath);
          String prefix = bundle.getFeedDataPathPrefix();
- 
          HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
- 
          List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
          HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
      }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceSuspendTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
index b64fca4,c2d8c9b..743990e
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessLibPathTest.java
@@@ -55,35 -59,29 +59,29 @@@ public class ProcessLibPathTest extend
  
      @BeforeClass(alwaysRun = true)
      public void createTestData() throws Exception {
- 
          logger.info("in @BeforeClass");
+ 
          //common lib for both test cases
          HadoopUtil.uploadDir(clusterFS, testLibDir, OSUtil.RESOURCES_OOZIE + "lib");
- 
 -        Bundle b = BundleUtil.readELBundle();
 +        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          b.generateUniqueBundle();
          b = new Bundle(b, cluster);
- 
          String startDate = "2010-01-01T22:00Z";
          String endDate = "2010-01-02T03:00Z";
- 
-         b.setInputFeedDataPath(testDir + "/input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+         b.setInputFeedDataPath(testDir + "/input" + MINUTE_DATE_PATTERN);
          String prefix = b.getFeedDataPathPrefix();
          HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
- 
          List<String> dataDates = TimeUtil.getMinuteDatesOnEitherSide(startDate, endDate, 20);
- 
          HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
      }
  
      @BeforeMethod(alwaysRun = true)
      public void testName(Method method) throws Exception {
          logger.info("test name: " + method.getName());
 -        bundles[0] = BundleUtil.readELBundle();
 +        bundles[0] = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
          bundles[0] = new Bundle(bundles[0], cluster);
          bundles[0].generateUniqueBundle();
-         bundles[0].setInputFeedDataPath(baseHDFSDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+         bundles[0].setInputFeedDataPath(baseHDFSDir + MINUTE_DATE_PATTERN);
          bundles[0].setProcessValidity("2010-01-02T01:00Z", "2010-01-02T01:04Z");
          bundles[0].setProcessPeriodicity(5, TimeUnit.minutes);
          bundles[0].setOutputFeedPeriodicity(5, TimeUnit.minutes);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
index 0000000,8b5c01c..5598744
mode 000000,100644..100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
@@@ -1,0 -1,269 +1,269 @@@
+ /**
+  * 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.hcat;
+ 
+ import org.apache.falcon.entity.v0.EntityType;
+ import org.apache.falcon.entity.v0.Frequency;
+ import org.apache.falcon.entity.v0.cluster.Interfacetype;
+ import org.apache.falcon.entity.v0.feed.ActionType;
+ import org.apache.falcon.entity.v0.feed.ClusterType;
+ import org.apache.falcon.regression.Entities.FeedMerlin;
+ import org.apache.falcon.regression.core.bundle.Bundle;
+ import org.apache.falcon.regression.core.enumsAndConstants.FeedType;
+ 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.HCatUtil;
+ import org.apache.falcon.regression.core.util.OSUtil;
+ import org.apache.falcon.regression.core.util.Util;
+ import org.apache.falcon.regression.core.util.InstanceUtil;
+ import org.apache.falcon.regression.core.util.XmlUtil;
+ import org.apache.falcon.regression.testHelper.BaseTestClass;
+ import org.apache.hive.hcatalog.api.HCatClient;
+ import org.apache.hive.hcatalog.api.HCatCreateTableDesc;
+ import org.apache.hive.hcatalog.common.HCatException;
+ import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
+ import org.apache.log4j.Logger;
+ import org.apache.oozie.client.Job;
+ import org.apache.oozie.client.OozieClient;
+ import org.testng.Assert;
+ import org.testng.annotations.BeforeClass;
+ import org.testng.annotations.BeforeMethod;
+ import org.testng.annotations.AfterMethod;
+ import org.testng.annotations.AfterClass;
+ import org.testng.annotations.Test;
+ 
+ import java.io.IOException;
+ import java.lang.reflect.Method;
+ import java.util.ArrayList;
+ import java.util.List;
+ 
+ public class HCatFeedOperationsTest extends BaseTestClass {
+ 
+     ColoHelper cluster = servers.get(0);
+     OozieClient clusterOC = serverOC.get(0);
+     HCatClient clusterHC = cluster.getClusterHelper().getHCatClient();
+ 
+     ColoHelper cluster2 = servers.get(1);
+     OozieClient cluster2OC = serverOC.get(1);
+     HCatClient cluster2HC = cluster2.getClusterHelper().getHCatClient();
+ 
+     private String dbName = "default";
+     private String tableName = "hcatFeedOperationsTest";
+     private String randomTblName = "randomTable_HcatFeedOperationsTest";
+     private String feed;
+     private String aggregateWorkflowDir = baseHDFSDir + "/HCatFeedOperationsTest/aggregator";
+     private static final Logger LOGGER = Logger.getLogger(HCatFeedOperationsTest.class);
+ 
+     public void uploadWorkflow() throws Exception {
+         uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
+     }
+ 
+     @BeforeClass(alwaysRun = true)
+     public void createTestData() throws Exception {
+         //create an empty table for feed operations
+         ArrayList<HCatFieldSchema> partitions = new ArrayList<HCatFieldSchema>();
+         partitions.add(HCatUtil.getStringSchema("year", "yearPartition"));
+         createEmptyTable(clusterHC, dbName, tableName, partitions);
+ 
+         //A random table to test submission of replication feed when table doesn't exist on target
+         createEmptyTable(clusterHC, dbName, randomTblName, partitions);
+ 
+         //create empty table on target cluster
+         createEmptyTable(cluster2HC, dbName, tableName, new ArrayList<HCatFieldSchema>());
+     }
+ 
+     @BeforeMethod(alwaysRun = true)
+     public void setUp(Method method) throws Exception {
+         LOGGER.info("test name: " + method.getName());
 -        Bundle bundle = BundleUtil.readHCatBundle();
++        Bundle bundle = BundleUtil.readHCatBundle(baseAppHDFSDir, this.getClass().getSimpleName());
+         bundles[0] = new Bundle(bundle, cluster.getPrefix());
+         bundles[0].generateUniqueBundle();
+         bundles[0].setClusterInterface(Interfacetype.REGISTRY, cluster.getClusterHelper().getHCatEndpoint());
+ 
+ 
+         bundles[1] = new Bundle(bundle, cluster2.getPrefix());
+         bundles[1].generateUniqueBundle();
+         bundles[1].setClusterInterface(Interfacetype.REGISTRY, cluster2.getClusterHelper().getHCatEndpoint());
+     }
+ 
+     @AfterMethod(alwaysRun = true)
+     public void tearDown() throws HCatException {
+         removeBundles();
+     }
+ 
+     @AfterClass(alwaysRun = true)
+     public void tearDownClass() throws IOException {
+         clusterHC.dropTable(dbName, tableName, true);
+         clusterHC.dropTable(dbName, randomTblName, true);
+         cluster2HC.dropTable(dbName, tableName, true);
+         cleanTestDirs();
+     }
+ 
+     /**
+      * Submit Hcat feed when Hcat table mentioned in table uri does not exist. Response should reflect failure.
+      *
+      * @throws Exception
+      */
+     @Test(groups = {"singleCluster"})
+     public void submitFeedWhenTableDoesNotExist() throws Exception {
+         Bundle.submitCluster(bundles[1]);
+         feed = bundles[1].getInputFeedFromBundle();
+         FeedMerlin feedObj = new FeedMerlin(feed);
+         feedObj.setTableValue(dbName, randomTblName, FeedType.YEARLY.getHcatPathValue());
+         ServiceResponse response = prism.getFeedHelper().submitEntity(feedObj.toString());
+         AssertUtil.assertFailed(response);
+     }
+ 
+     /**
+      * Submit Hcat feed when Hcat table mentioned in table uri exists. Delete that feed, and re-submit.
+      * All responses should reflect success.
+      *
+      * @throws Exception
+      */
+     @Test(groups = {"singleCluster"})
+     public void submitFeedPostDeletionWhenTableExists() throws Exception {
+         Bundle.submitCluster(bundles[0]);
+         feed = bundles[0].getInputFeedFromBundle();
+         FeedMerlin feedObj = new FeedMerlin(feed);
+         feedObj.setTableValue(dbName, tableName, FeedType.YEARLY.getHcatPathValue());
+         ServiceResponse response = prism.getFeedHelper().submitEntity(feedObj.toString());
+         AssertUtil.assertSucceeded(response);
+ 
+         response = prism.getFeedHelper().delete(feedObj.toString());
+         AssertUtil.assertSucceeded(response);
+ 
+         response = prism.getFeedHelper().submitEntity(feedObj.toString());
+         AssertUtil.assertSucceeded(response);
+     }
+ 
+     /**
+      * Submit Hcat Replication feed when Hcat table mentioned in table uri does not exist on target. The response is
+      * Partial, with successful with submit/schedule on source.
+      *
+      * @throws Exception
+      */
+     @Test
+     public void submitAndScheduleReplicationFeedWhenTableDoesNotExistOnTarget() throws Exception {
+         Bundle.submitCluster(bundles[0], bundles[1]);
+         final String startDate = "2010-01-01T20:00Z";
+         final String endDate = "2099-01-01T00:00Z";
+         String tableUri = "catalog:" + dbName + ":" + randomTblName + "#year=${YEAR}";
+         bundles[0].setInputFeedPeriodicity(1, Frequency.TimeUnit.hours);
+         bundles[0].setInputFeedValidity(startDate, endDate);
+         bundles[0].setInputFeedTableUri(tableUri);
+ 
+         feed = bundles[0].getDataSets().get(0);
+         // set cluster 2 as the target.
+         feed = InstanceUtil.setFeedClusterWithTable(feed,
+                 XmlUtil.createValidity(startDate, endDate),
+                 XmlUtil.createRtention("months(9000)", ActionType.DELETE),
+                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
+                 tableUri);
+ 
+         AssertUtil.assertPartial(prism.getFeedHelper().submitAndSchedule(feed));
+     }
+ 
+     /**
+      * Submit Hcat Replication feed when Hcat table mentioned in table uri exists on both source and target. The response is
+      * Psucceeded, and a replication co-rdinator should apear on target oozie. The test however does not ensure that
+      * replication goes through.
+      *
+      * @throws Exception
+      */
+     @Test
+     public void submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget() throws Exception {
+         Bundle.submitCluster(bundles[0], bundles[1]);
+         final String startDate = "2010-01-01T20:00Z";
+         final String endDate = "2099-01-01T00:00Z";
+         String tableUri = "catalog:" + dbName + ":" + tableName + "#year=${YEAR}";
+         bundles[0].setInputFeedPeriodicity(1, Frequency.TimeUnit.hours);
+         bundles[0].setInputFeedValidity(startDate, endDate);
+         bundles[0].setInputFeedTableUri(tableUri);
+ 
+         feed = bundles[0].getDataSets().get(0);
+         // set cluster 2 as the target.
+         feed = InstanceUtil.setFeedClusterWithTable(feed,
+                 XmlUtil.createValidity(startDate, endDate),
+                 XmlUtil.createRtention("months(9000)", ActionType.DELETE),
+                 Util.readEntityName(bundles[1].getClusters().get(0)), ClusterType.TARGET, null,
+                 tableUri);
+ 
+         AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed));
+         Assert.assertEquals(InstanceUtil
+                 .checkIfFeedCoordExist(cluster2.getFeedHelper(), Util.readEntityName(feed),
+                         "REPLICATION"), 1);
+         //This test doesn't wait for replication to succeed.
+     }
+ 
+     /**
+      * Submit Hcat Replication feed. Suspend the feed, and check that feed was suspended on
+      * both clusters. Now resume feed, and check that status is running on both clusters.
+      * The test however does not ensure that replication goes through.
+      *
+      * @throws Exception
+      */
+     @Test
+     public void suspendAndResumeReplicationFeed() throws Exception {
+ 
+         submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
+ 
+         AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(feed));
+ 
+         //check that feed suspended on both clusters
+         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
+         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.SUSPENDED);
+ 
+         AssertUtil.assertSucceeded(prism.getFeedHelper().resume(feed));
+ 
+         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
+         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.RUNNING);
+     }
+ 
+     /**
+      * Submit Hcat Replication feed. Delete the feed, and check that feed was deleted on
+      * both clusters. The test however does not ensure that replication goes through.
+      *
+      * @throws Exception
+      */
+     @Test
+     public void deleteReplicationFeed() throws Exception {
+         submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
+ 
+         AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed));
+         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.KILLED);
+         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, feed, Job.Status.KILLED);
+     }
+ 
+ 
+     public static void createEmptyTable(HCatClient cli, String dbName, String tabName, List<HCatFieldSchema> partitionCols) throws HCatException{
+ 
+         ArrayList<HCatFieldSchema> cols = new ArrayList<HCatFieldSchema>();
+         cols.add(HCatUtil.getStringSchema("id", "id comment"));
+         HCatCreateTableDesc tableDesc = HCatCreateTableDesc
+                 .create(dbName, tabName, cols)
+                 .partCols(partitionCols)
+                 .fileFormat("textfile")
+                 .ifNotExists(true)
+                 .isTableExternal(true)
+                 .build();
+         cli.createTable(tableDesc);
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatProcessTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatRetentionTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiProcessInstanceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/LineageApiTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/EntityDryRunTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedDelayParallelTimeoutTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
index 25a8789,0000000..e69de29
mode 100644,000000..100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/FeedRetentionTest.java

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/OptionalInputTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSnSTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedSuspendTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessDeleteTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessResumeTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/ProcessPartitionExpVariableTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
index 233c50f,378cbbd..71af1e2
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
@@@ -80,19 -79,17 +79,18 @@@ public class UpdateAtSpecificTimeTest e
      @BeforeMethod(alwaysRun = true)
      public void setup(Method method) throws IOException {
          logger.info("test name: " + method.getName());
 -        Bundle bundle = BundleUtil.readLocalDCBundle();
 +        Bundle bundle =
 +            BundleUtil.readLocalDCBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-         bundles[0] = new Bundle(bundle, cluster_1);
-         bundles[1] = new Bundle(bundle, cluster_2);
-         bundles[2] = new Bundle(bundle, cluster_3);
+         bundles[0] = new Bundle(bundle, cluster1);
+         bundles[1] = new Bundle(bundle, cluster2);
+         bundles[2] = new Bundle(bundle, cluster3);
  
          bundles[0].generateUniqueBundle();
          bundles[1].generateUniqueBundle();
          bundles[2].generateUniqueBundle();
  
 -        processBundle = BundleUtil.readELBundle();
 +        processBundle = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-         processBundle = new Bundle(processBundle, cluster_1);
+         processBundle = new Bundle(processBundle, cluster1);
          processBundle.generateUniqueBundle();
          processBundle.setProcessWorkflow(aggregateWorkflowDir);
      }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
index b725b14,d7ac724..b014dad
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
@@@ -62,9 -62,7 +62,8 @@@ public class LineageGraphTest extends B
      private String baseTestDir = baseHDFSDir + "/LineageGraphTest";
      private String aggregateWorkflowDir = baseTestDir + "/aggregator";
      private static final Logger logger = Logger.getLogger(LineageGraphTest.class);
-     String datePattern = "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
-     String feedInputPath = baseTestDir + "/input" + datePattern;
-     String feedOutputPath = baseTestDir + "/output" + datePattern;
 -    String feedInputPath = baseTestDir + "/input" + MINUTE_DATE_PATTERN;
++    String feedInputPath = baseTestDir + "/input" + MINUTE_DATE_PATTERN;;
++    String feedOutputPath = baseTestDir + "/output" + MINUTE_DATE_PATTERN;;
      private FileSystem clusterFS = serverFS.get(0);
      private OozieClient clusterOC = serverOC.get(0);
      private String processName = null;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/0d3e97d7/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
----------------------------------------------------------------------
diff --cc falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
index 6c9c27f,76173b7..6f673c9
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
@@@ -95,8 -95,7 +95,8 @@@ public class ProcessUITest extends Base
          bundles[0].setProcessPeriodicity(1, Frequency.TimeUnit.minutes);
          bundles[0].setProcessConcurrency(5);
          bundles[0].setInputFeedPeriodicity(1, Frequency.TimeUnit.minutes);
-         bundles[0].setInputFeedDataPath(feedInputPath + datePattern);
-         bundles[0].setOutputFeedLocationData(feedOutputPath + datePattern);
+         bundles[0].setInputFeedDataPath(feedInputPath + MINUTE_DATE_PATTERN);
++        bundles[0].setOutputFeedLocationData(feedOutputPath + MINUTE_DATE_PATTERN);
          Process process = bundles[0].getProcessObject();
          Inputs inputs = new Inputs();
          Input input = new Input();


[37/41] FALCON-675 Request URLS moved from parameters into methods in falcon-regression

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
index 9198901..db29b19 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/NewPrismProcessUpdateTest.java
@@ -36,7 +36,6 @@ import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -130,8 +129,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
@@ -183,8 +181,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -257,8 +254,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
@@ -342,8 +338,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
@@ -382,8 +377,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String originalProcessData = bundles[1].getProcessData();
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
@@ -417,8 +411,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
 
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
@@ -500,8 +493,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -587,14 +579,12 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
         waitForProcessToReachACertainState(cluster3, bundles[1], Job.Status.RUNNING);
@@ -602,8 +592,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
                 EntityType.PROCESS);
 
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().suspend(bundles[1].getProcessData()));
         //now to update
         while (Util
                 .parseResponse(updateProcessConcurrency(bundles[1],
@@ -622,8 +611,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         OozieUtil.verifyNewBundleCreation(cluster3, oldBundleId, oldNominalTimes,
                 bundles[1].getProcessData(), false, true);
         AssertUtil.checkNotStatus(cluster2OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(cluster3.getProcessHelper()
-                .resume(URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster3.getProcessHelper().resume(bundles[1].getProcessData()));
         AssertUtil.checkStatus(cluster3OC, EntityType.PROCESS, bundles[1], Job.Status.RUNNING);
 
         Job.Status status = OozieUtil.getOozieJobStatus(cluster3.getFeedHelper().getOozieClient(),
@@ -682,8 +670,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         LOGGER.info("process to be scheduled: " + Util.prettyPrintXml(bundles[1].getProcessData()));
 
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
         String oldBundleId = InstanceUtil
@@ -783,8 +770,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -852,8 +838,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -873,8 +858,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].getProcessObject().setOrder(getRandomExecutionType(bundles[1]));
         //suspend
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().suspend(bundles[1].getProcessData()));
 
         //now to update
         String updateTime = new DateTime(DateTimeZone.UTC).plusMinutes(2).toString();
@@ -886,8 +870,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
             TimeUtil.sleepSeconds(10);
         }
 
-        AssertUtil.assertSucceeded(cluster3.getProcessHelper()
-                .resume(URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster3.getProcessHelper().resume(bundles[1].getProcessData()));
 
         String prismString = getResponse(prism, bundles[1].getProcessData(), true);
         Assert.assertEquals(Util.getProcessObject(prismString).getParallel(),
@@ -929,8 +912,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -950,8 +932,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         inputFeed = Util.setFeedName(inputFeed, newFeedName);
 
         LOGGER.info(inputFeed);
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, inputFeed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(inputFeed));
 
         while (Util.parseResponse(
                 prism.getProcessHelper()
@@ -983,8 +964,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0);
 
@@ -1003,10 +983,8 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         inputFeed = Util.setFeedName(inputFeed, newFeedName);
 
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, inputFeed));
+                cluster3.getProcessHelper().suspend(bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(inputFeed));
 
         while (Util.parseResponse(
                 prism.getProcessHelper()
@@ -1019,8 +997,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
                 bundles[1].getProcessData(), true, false);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 1, 10);
 
-        AssertUtil.assertSucceeded(cluster3.getProcessHelper()
-                .resume(URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster3.getProcessHelper().resume(bundles[1].getProcessData()));
 
         bundles[1].verifyDependencyListing(cluster2);
 
@@ -1050,8 +1027,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
 
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, originalProcess));
+                cluster3.getProcessHelper().schedule(originalProcess));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, originalProcess, 0, 10);
 
@@ -1064,8 +1040,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
                 EntityType.PROCESS);
 
         //submit new feed
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, inputFeed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(inputFeed));
 
         Util.shutDownService(cluster3.getProcessHelper());
 
@@ -1114,8 +1089,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -1181,8 +1155,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
@@ -1200,8 +1173,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
             newEndTime);
 
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().suspend(bundles[1].getProcessData()));
         while (Util.parseResponse(
                 prism.getProcessHelper()
                         .update((bundles[1].getProcessData()), bundles[1].getProcessData()))
@@ -1209,8 +1181,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
             LOGGER.info("update didnt SUCCEED in last attempt");
             TimeUtil.sleepSeconds(10);
         }
-        AssertUtil.assertSucceeded(cluster3.getProcessHelper()
-                .resume(URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster3.getProcessHelper().resume(bundles[1].getProcessData()));
 
         dualComparison(prism, cluster2, bundles[1].getProcessData());
 
@@ -1249,8 +1220,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
         String oldBundleId = InstanceUtil
@@ -1301,8 +1271,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -1346,8 +1315,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         TimeUtil.sleepSeconds(30);
         InstanceUtil.waitTillInstancesAreCreated(cluster3, bundles[1].getProcessData(), 0, 10);
 
@@ -1385,9 +1353,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData())
-        );
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
@@ -1410,9 +1376,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         waitForProcessToReachACertainState(cluster3, bundles[1], Job.Status.RUNNING);
 
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData())
-        );
+                cluster3.getProcessHelper().suspend(bundles[1].getProcessData()));
 
         AssertUtil.assertSucceeded(
                 prism.getProcessHelper()
@@ -1447,8 +1411,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         bundles[1].submitBundle(prism);
         //now to schedule in 1 colo and let it remain in another
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .schedule(URLS.SCHEDULE_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().schedule(bundles[1].getProcessData()));
         String oldBundleId = InstanceUtil
                 .getLatestBundleID(cluster3,
                         Util.readEntityName(bundles[1].getProcessData()), EntityType.PROCESS);
@@ -1467,13 +1430,11 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
         waitForProcessToReachACertainState(cluster3, bundles[1], Job.Status.RUNNING);
 
         AssertUtil.assertSucceeded(
-                cluster3.getProcessHelper()
-                        .suspend(URLS.SUSPEND_URL, bundles[1].getProcessData()));
+                cluster3.getProcessHelper().suspend(bundles[1].getProcessData()));
         AssertUtil.assertSucceeded(
                 prism.getProcessHelper()
                         .update(bundles[1].getProcessData(), bundles[1].getProcessData()));
-        AssertUtil.assertSucceeded(cluster3.getProcessHelper()
-                .resume(URLS.RESUME_URL, bundles[1].getProcessData()));
+        AssertUtil.assertSucceeded(cluster3.getProcessHelper().resume(bundles[1].getProcessData()));
         List<String> oldNominalTimes =
                 OozieUtil.getActionsNominalTime(cluster3, oldBundleId, EntityType.PROCESS);
 
@@ -1579,8 +1540,7 @@ public class NewPrismProcessUpdateTest extends BaseTestClass {
 
     private String getResponse(ColoHelper prism, String processData, boolean bool)
         throws Exception {
-        ServiceResponse response = prism.getProcessHelper()
-                .getEntityDefinition(Util.URLS.GET_ENTITY_DEFINITION, processData);
+        ServiceResponse response = prism.getProcessHelper().getEntityDefinition(processData);
         if (bool) {
             AssertUtil.assertSucceeded(response);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
index a4c2dae..c601345 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismClusterDeleteTest.java
@@ -75,7 +75,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
     @Test(groups = {"multiCluster"})
     public void testServer1ClusterDeleteInBothColos() throws Exception {
         AssertUtil.assertSucceeded((prism.getClusterHelper()
-            .submitEntity(Util.URLS.SUBMIT_URL, bundles[0].getClusters().get(0))));
+            .submitEntity(bundles[0].getClusters().get(0))));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -90,7 +90,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
 
         //lets now delete the cluster from both colos
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -127,7 +127,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
         restartRequired = true;
 
         AssertUtil.assertSucceeded(prism.getClusterHelper()
-            .submitEntity(Util.URLS.SUBMIT_URL, bundles[0].getClusters().get(0)));
+            .submitEntity(bundles[0].getClusters().get(0)));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -146,7 +146,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
 
         //lets now delete the cluster from both colos
         AssertUtil.assertPartial(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -177,7 +177,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
         //now bring up the service and roll forward the delete
         Util.startService(cluster1.getClusterHelper());
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //get final data states:
         List<String> server1ArchiveFinalState2 = cluster1.getClusterHelper().getArchiveInfo();
@@ -204,9 +204,9 @@ public class PrismClusterDeleteTest extends BaseTestClass {
     public void testServer1ClusterDeleteAlreadyDeletedCluster() throws Exception {
         restartRequired = true;
         AssertUtil.assertSucceeded(prism.getClusterHelper()
-            .submitEntity(Util.URLS.SUBMIT_URL, bundles[0].getClusters().get(0)));
+            .submitEntity(bundles[0].getClusters().get(0)));
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -220,7 +220,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
         List<String> initialServer2ArchiveStore = cluster2.getClusterHelper().getArchiveInfo();
 
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -255,12 +255,12 @@ public class PrismClusterDeleteTest extends BaseTestClass {
         restartRequired = true;
 
         AssertUtil.assertSucceeded(prism.getClusterHelper()
-            .submitEntity(Util.URLS.SUBMIT_URL, bundles[0].getClusters().get(0)));
+            .submitEntity(bundles[0].getClusters().get(0)));
         Util.shutDownService(cluster1.getClusterHelper());
 
         //lets now delete the cluster from both colos
         AssertUtil.assertPartial(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //now lets get the final states
         List<String> initialPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -278,7 +278,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
 
         //delete again
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //get final states
         List<String> finalPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -323,7 +323,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
 
         //delete
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //get final states
         List<String> finalPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -374,7 +374,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
 
         //delete
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         //get final states
         List<String> finalPrismStore = prism.getClusterHelper().getStoreInfo();
@@ -404,7 +404,7 @@ public class PrismClusterDeleteTest extends BaseTestClass {
 
         Util.startService(cluster1.getFeedHelper());
         AssertUtil.assertSucceeded(
-            prism.getClusterHelper().delete(Util.URLS.DELETE_URL, bundles[0].getClusters().get(0)));
+            prism.getClusterHelper().delete(bundles[0].getClusters().get(0)));
 
         List<String> server1StorePostUp = cluster1.getClusterHelper().getStoreInfo();
         List<String> server1ArchivePostUp = cluster1.getClusterHelper().getArchiveInfo();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
index 630ca28..c767649 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismConcurrentRequestTest.java
@@ -80,7 +80,7 @@ public class PrismConcurrentRequestTest extends BaseTestClass {
      */
     @Test(groups = {"multiCluster"})
     public void submitFeedParallel() throws Exception {
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         for (int i = 1; i <= brothers.length; i++) {
             brothers[i - 1] =
                 new Brother("brother" + i, "submit", EntityType.FEED, brotherGrimm, bundles[0],
@@ -104,9 +104,9 @@ public class PrismConcurrentRequestTest extends BaseTestClass {
      */
     @Test(groups = {"multiCluster"})
     public void submitProcessParallel() throws Exception {
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
-        prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(0));
-        prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getDataSets().get(1));
+        prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
+        prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(0));
+        prism.getFeedHelper().submitEntity(bundles[0].getDataSets().get(1));
         for (int i = 1; i <= brothers.length; i++) {
             brothers[i - 1] =
                 new Brother("brother" + i, "submit", EntityType.PROCESS, brotherGrimm, bundles[0],
@@ -181,11 +181,11 @@ public class PrismConcurrentRequestTest extends BaseTestClass {
     @Test(groups = {"multiCluster"})
     public void resumeAndSuspendFeedParallel() throws Exception {
         brothers = new Brother[4];
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
-        prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
+        prism.getFeedHelper().submitEntity(feed);
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
-        prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        prism.getFeedHelper().suspend(feed);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
         for (int i = 1; i <= 2; i++) {
             brothers[i - 1] =
@@ -217,15 +217,15 @@ public class PrismConcurrentRequestTest extends BaseTestClass {
     @Test(groups = {"multiCluster"})
     public void resumeFeedParallel() throws Exception {
         final double delay = 15;
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
-        prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
+        prism.getFeedHelper().submitEntity(feed);
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         TimeUtil.sleepSeconds(delay);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
-        prism.getFeedHelper().resume(URLS.RESUME_URL, feed);
+        prism.getFeedHelper().resume(feed);
         TimeUtil.sleepSeconds(delay);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.RUNNING);
-        prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        prism.getFeedHelper().suspend(feed);
         TimeUtil.sleepSeconds(delay);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, feed, Job.Status.SUSPENDED);
         for (int i = 1; i <= brothers.length; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
index 9e3ed12..f67e2ce 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedDeleteTest.java
@@ -32,7 +32,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
@@ -107,8 +106,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -160,8 +158,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         Util.shutDownService(cluster1.getFeedHelper());
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -192,8 +189,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         Util.startService(cluster1.getFeedHelper());
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         List<String> Server2ArchivePostUp = cluster2.getFeedHelper().getArchiveInfo();
         List<String> Server2StorePostUp = cluster2.getFeedHelper().getStoreInfo();
@@ -219,8 +215,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
     public void testServer1FeedDeleteAlreadyDeletedFeed() throws Exception {
         restartRequired = true;
         bundles[0].submitFeed();
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -233,8 +228,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2Store = cluster2.getFeedHelper().getStoreInfo();
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -274,8 +268,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> initialPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -292,8 +285,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         Util.startService(cluster1.getFeedHelper());
 
         //delete again
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //get final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -337,8 +329,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2Store = cluster2.getFeedHelper().getStoreInfo();
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //get final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -377,13 +368,12 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[0].setCLusterColo(cluster1Colo);
         logger.info("cluster bundle1: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
 
-        ServiceResponse r = prism.getClusterHelper()
-            .submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         bundles[1].setCLusterColo(cluster2Colo);
         logger.info("cluster bundle2: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[1].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         String startTimeServer1 = "2012-10-01T12:00Z";
@@ -409,7 +399,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(Util.URLS.DELETE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed));
     }
 
 
@@ -430,8 +420,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -469,8 +458,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[1].submitAndScheduleFeed();
 
         //suspend Server1 colo thingy
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -484,8 +472,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -524,10 +511,8 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[1].submitAndScheduleFeed();
 
         //suspend Server1 colo thingy
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -541,8 +526,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -580,8 +564,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
 
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -598,8 +581,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         Util.shutDownService(cluster1.getFeedHelper());
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -630,8 +612,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         Util.startService(cluster1.getClusterHelper());
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         List<String> Server1StorePostUp = cluster1.getFeedHelper().getStoreInfo();
         List<String> Server1ArchivePostUp = cluster1.getFeedHelper().getArchiveInfo();
@@ -662,10 +643,8 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeed();
         bundles[1].submitAndScheduleFeed();
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
         //fetch the initial store and archive state for prism
         List<String> initialPrismStore = prism.getFeedHelper().getStoreInfo();
         List<String> initialPrismArchiveStore = prism.getFeedHelper().getArchiveInfo();
@@ -681,8 +660,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         Util.shutDownService(cluster1.getFeedHelper());
 
         //lets now delete the feed from both colos
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -713,8 +691,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         Util.startService(cluster1.getFeedHelper());
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         HashMap<String, List<String>> finalSystemState = getSystemState(EntityType.FEED);
 
@@ -754,8 +731,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         Util.shutDownService(cluster1.getFeedHelper());
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -786,8 +762,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
 
         Util.startService(cluster1.getClusterHelper());
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         HashMap<String, List<String>> systemStatePostUp = getSystemState(EntityType.FEED);
 
@@ -812,13 +787,12 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[0].setCLusterColo(cluster1Colo);
         logger.info("cluster bundle1: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         bundles[1].setCLusterColo(cluster2Colo);
         logger.info("cluster bundle2: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[1].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         String startTimeServer1 = "2012-10-01T12:00Z";
@@ -844,11 +818,11 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        r = prism.getFeedHelper().submitEntity(feed);
 
         AssertUtil.assertSucceeded(r);
 
-        r = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(r);
         TimeUtil.sleepSeconds(15);
 
@@ -865,14 +839,14 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        r = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        r = prism.getFeedHelper().suspend(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertPartial(r);
         Assert
             .assertTrue(r.getMessage().contains(cluster1Colo + "/org.apache.falcon.FalconException")
                 && r.getMessage().contains(cluster2Colo + "/" + Util.readEntityName(feed)));
 
-        ServiceResponse response = prism.getFeedHelper().delete(Util.URLS.DELETE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().delete(feed);
         Assert.assertTrue(
             response.getMessage().contains(cluster1Colo + "/org.apache.falcon.FalconException")
                 && response.getMessage().contains(cluster2Colo + "/" + Util.readEntityName(feed)));
@@ -914,13 +888,12 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         bundles[0].setCLusterColo(cluster1Colo);
         logger.info("cluster bundle1: " + Util.prettyPrintXml(bundles[0].getClusters().get(0)));
 
-        ServiceResponse r =
-            prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[0].getClusters().get(0));
+        ServiceResponse r = prism.getClusterHelper().submitEntity(bundles[0].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         bundles[1].setCLusterColo(cluster2Colo);
         logger.info("cluster bundle2: " + Util.prettyPrintXml(bundles[1].getClusters().get(0)));
-        r = prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
+        r = prism.getClusterHelper().submitEntity(bundles[1].getClusters().get(0));
         Assert.assertTrue(r.getMessage().contains("SUCCEEDED"));
 
         String startTimeServer1 = "2012-10-01T12:00Z";
@@ -944,11 +917,11 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        r = prism.getFeedHelper().submitEntity(feed);
 
         AssertUtil.assertSucceeded(r);
 
-        r = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(r);
         TimeUtil.sleepSeconds(15);
 
@@ -963,13 +936,13 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         List<String> initialServer2Store = cluster2.getFeedHelper().getStoreInfo();
         List<String> initialServer2ArchiveStore = cluster2.getFeedHelper().getArchiveInfo();
 
-        r = prism.getFeedHelper().suspend(URLS.SUSPEND_URL, feed);
+        r = prism.getFeedHelper().suspend(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        ServiceResponse response = prism.getFeedHelper().delete(Util.URLS.DELETE_URL, feed);
+        ServiceResponse response = prism.getFeedHelper().delete(feed);
         Assert.assertTrue(response.getMessage().contains(cluster1Colo + "/org.apache.falcon" +
             ".FalconException")
             && response.getMessage().contains(cluster2Colo + "/" + Util.readEntityName(feed)));
@@ -1026,8 +999,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
         Util.shutDownService(cluster1.getFeedHelper());
 
         //lets now delete the cluster from both colos
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[1].getDataSets().get(0)));
 
         //now lets get the final states
         List<String> finalPrismStore = prism.getFeedHelper().getStoreInfo();
@@ -1058,8 +1030,7 @@ public class PrismFeedDeleteTest extends BaseTestClass {
 
         Util.startService(cluster1.getFeedHelper());
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         clusterName = Util.readEntityName(bundles[0].getDataSets().get(0));
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
index 8a14229..9e0fbed 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedLateReplicationTest.java
@@ -29,7 +29,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -127,7 +126,7 @@ public class PrismFeedLateReplicationTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
 
         String bundleId =
@@ -200,7 +199,7 @@ public class PrismFeedLateReplicationTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
 
         String postFix = "/US/" + cluster2.getClusterHelper().getColoName();
@@ -393,7 +392,7 @@ public class PrismFeedLateReplicationTest extends BaseTestClass {
         //submit and schedule feed
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
 
         //wait till 1st instance of replication coord is SUCCEEDED
@@ -595,7 +594,7 @@ public class PrismFeedLateReplicationTest extends BaseTestClass {
         //submit and schedule feed
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
 
         //wait till 1st instance of replication coord is SUCCEEDED

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
index 2d67583..7a5a91e 100755
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationPartitionExpTest.java
@@ -31,7 +31,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -230,7 +229,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertFailed(r, "submit of feed should have failed as the partition in source " +
             "is blank");
@@ -278,11 +277,11 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
-        r = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(r);
         TimeUtil.sleepSeconds(15);
 
@@ -373,8 +372,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
@@ -466,11 +464,11 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         TimeUtil.sleepSeconds(15);
 
         InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(feed), 1,
@@ -563,7 +561,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
         //clean target if old data exists
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         AssertUtil.assertFailed(r, "Submission of feed should have failed.");
         Assert.assertTrue(r.getMessage().contains(
                 "Partition expression has to be specified for cluster " +
@@ -619,11 +617,11 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         TimeUtil.sleepSeconds(15);
 
         InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(feed), 1,
@@ -720,11 +718,11 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
-        r = prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed);
+        r = prism.getFeedHelper().schedule(feed);
         AssertUtil.assertSucceeded(r);
         TimeUtil.sleepSeconds(15);
 
@@ -802,11 +800,11 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
         TimeUtil.sleepSeconds(15);
         InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(feed), 1,
             CoordinatorAction.Status.SUCCEEDED, EntityType.FEED, 20);
@@ -890,7 +888,7 @@ public class PrismFeedReplicationPartitionExpTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        ServiceResponse r = prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitEntity(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertFailed(r, "is defined more than once for feed");
         Assert.assertTrue(r.getMessage().contains("is defined more than once for feed"));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
index a4c27d7..56e0af9 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
@@ -30,7 +30,6 @@ import org.apache.falcon.regression.core.util.InstanceUtil;
 import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
 import org.apache.falcon.regression.core.util.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.fs.FileSystem;
@@ -142,8 +141,8 @@ public class PrismFeedReplicationUpdateTest extends BaseTestClass {
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, feed));
-        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(URLS.SCHEDULE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().schedule(feed));
 
         //change feed location path
         feed = InstanceUtil.setFeedFilePath(feed, alternativeInputPath);
@@ -264,9 +263,9 @@ public class PrismFeedReplicationUpdateTest extends BaseTestClass {
             Util.readEntityName(bundles[2].getClusters().get(0)), ClusterType.TARGET, null);
 
         //submit and schedule feeds
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed01);
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed02);
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_URL, outputFeed);
+        prism.getFeedHelper().submitAndSchedule(feed01);
+        prism.getFeedHelper().submitAndSchedule(feed02);
+        prism.getFeedHelper().submitAndSchedule(outputFeed);
 
         //create a process with 2 clusters
 
@@ -291,8 +290,7 @@ public class PrismFeedReplicationUpdateTest extends BaseTestClass {
             Util.readEntityName(feed02));
 
         //submit and schedule process
-        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(URLS
-            .SUBMIT_AND_SCHEDULE_URL, process));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(process));
 
         logger.info("Wait till process goes into running ");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
index af1620a..79ceacc 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedResumeTest.java
@@ -82,38 +82,36 @@ public class PrismFeedResumeTest extends BaseTestClass {
         bundles[1].submitAndScheduleFeedUsingColoHelper(cluster2);
 
         //suspend using prismHelper
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
         //verify
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.SUSPENDED);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         //suspend using prismHelper
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         //verify
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         //try using the colohelper
         AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+            .suspend(bundles[0].getDataSets().get(0)));
         //verify
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.SUSPENDED);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         //suspend using prismHelper
         AssertUtil.assertSucceeded(
-            cluster1.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+            cluster1.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         //verify
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
 
         //suspend on the other one
         AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+            .suspend(bundles[0].getDataSets().get(0)));
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.SUSPENDED);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
 
         AssertUtil.assertSucceeded(
-            cluster1.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+            cluster1.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
     }
@@ -125,27 +123,21 @@ public class PrismFeedResumeTest extends BaseTestClass {
         bundles[1].submitAndScheduleFeedUsingColoHelper(cluster2);
 
         //delete using coloHelpers
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         //suspend using prismHelper
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[1].getDataSets().get(0)));
         //suspend on the other one
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
-        AssertUtil.assertFailed(
-            cluster1.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster1.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
-        AssertUtil.assertFailed(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
     }
 
@@ -155,25 +147,23 @@ public class PrismFeedResumeTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeedUsingColoHelper(cluster1);
         bundles[1].submitAndScheduleFeedUsingColoHelper(cluster2);
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.SUSPENDED);
         for (int i = 0; i < 2; i++) {
             //suspend using prismHelper
             AssertUtil.assertSucceeded(prism.getFeedHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+                .resume(bundles[0].getDataSets().get(0)));
             //verify
             checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
             checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         }
 
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[1].getDataSets().get(0)));
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.SUSPENDED);
 
         for (int i = 0; i < 2; i++) {
             AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+                .resume(bundles[0].getDataSets().get(0)));
             //verify
             checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
             checkAndCompareStatus(cluster2, bundles[1], Job.Status.SUSPENDED);
@@ -183,7 +173,7 @@ public class PrismFeedResumeTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //suspend on the other one
             AssertUtil.assertSucceeded(prism.getFeedHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+                .resume(bundles[1].getDataSets().get(0)));
             AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
             AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);
         }
@@ -191,7 +181,7 @@ public class PrismFeedResumeTest extends BaseTestClass {
         for (int i = 0; i < 2; i++) {
             //suspend on the other one
             AssertUtil.assertSucceeded(cluster2.getFeedHelper()
-                .resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+                .resume(bundles[1].getDataSets().get(0)));
             checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
             checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         }
@@ -199,15 +189,11 @@ public class PrismFeedResumeTest extends BaseTestClass {
 
     @Test
     public void testResumeNonExistentFeedOnBothColos() throws Exception {
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
-
-        AssertUtil.assertFailed(
-            cluster1.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
+
+        AssertUtil.assertFailed(cluster1.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
     }
 
     @Test
@@ -215,15 +201,11 @@ public class PrismFeedResumeTest extends BaseTestClass {
         bundles[0].submitFeed();
         bundles[1].submitFeed();
 
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
 
-        AssertUtil.assertFailed(
-            cluster1.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster1.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
     }
 
     @Test(groups = {"prism", "0.2"})
@@ -234,28 +216,26 @@ public class PrismFeedResumeTest extends BaseTestClass {
         bundles[0].submitAndScheduleFeedUsingColoHelper(cluster1);
         bundles[1].submitAndScheduleFeedUsingColoHelper(cluster2);
         AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+            .suspend(bundles[0].getDataSets().get(0)));
         AssertUtil.assertSucceeded(cluster2.getFeedHelper()
-            .suspend(Util.URLS.SUSPEND_URL, bundles[1].getDataSets().get(0)));
+            .suspend(bundles[1].getDataSets().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         //verify
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.SUSPENDED);
         //resume on the other one
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
 
         Util.startService(cluster1.getFeedHelper());
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.SUSPENDED);
         AssertUtil.assertSucceeded(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+            cluster2.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         AssertUtil.assertSucceeded(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+            prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
         checkAndCompareStatus(cluster1, bundles[0], Job.Status.RUNNING);
     }
@@ -269,36 +249,29 @@ public class PrismFeedResumeTest extends BaseTestClass {
         bundles[1].submitAndScheduleFeedUsingColoHelper(cluster2);
 
         //delete using prismHelper
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[0].getDataSets().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
 
         //suspend using prismHelper
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
 
         //suspend using prismHelper
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
         //verify
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         checkAndCompareStatus(cluster2, bundles[1], Job.Status.RUNNING);
-        AssertUtil.assertSucceeded(
-            prism.getFeedHelper().delete(Util.URLS.DELETE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(bundles[1].getDataSets().get(0)));
         //suspend on the other one
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
 
-        AssertUtil.assertFailed(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.KILLED);
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.KILLED);
     }
@@ -309,12 +282,9 @@ public class PrismFeedResumeTest extends BaseTestClass {
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
     }
 
     @Test(groups = {"prism", "0.2"})
@@ -326,12 +296,9 @@ public class PrismFeedResumeTest extends BaseTestClass {
 
         Util.shutDownService(cluster1.getFeedHelper());
 
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[0].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            prism.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
-        AssertUtil.assertFailed(
-            cluster2.getFeedHelper().resume(Util.URLS.RESUME_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[1].getDataSets().get(0)));
     }
 
 
@@ -341,16 +308,11 @@ public class PrismFeedResumeTest extends BaseTestClass {
             .checkStatus(coloHelper.getFeedHelper().getOozieClient(), EntityType.FEED, bundle,
                 expectedStatus);
         String entity = bundle.getDataSets().get(0);
-        Assert.assertEquals(
-            Util.parseResponse(prism.getFeedHelper().getStatus(Util.URLS.STATUS_URL, entity))
-                .getMessage(),
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(entity)).getMessage(),
             coloHelper.getFeedHelper().getColoName() + "/" + expectedStatus);
-        Assert.assertEquals(
-            Util.parseResponse(prism.getFeedHelper().getStatus(Util.URLS.STATUS_URL, entity))
-                .getMessage(),
+        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper().getStatus(entity)).getMessage(),
             coloHelper.getFeedHelper().getColoName() + "/"
-                + Util.parseResponse(
-                coloHelper.getFeedHelper().getStatus(Util.URLS.STATUS_URL, entity)).getMessage());
+                + Util.parseResponse(coloHelper.getFeedHelper().getStatus(entity)).getMessage());
     }
 
     @AfterClass(alwaysRun = true)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/051a3d29/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
index ab2e95d..9a32932 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
@@ -24,7 +24,6 @@ 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.Util;
-import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
@@ -81,8 +80,7 @@ public class PrismFeedScheduleTest extends BaseTestClass {
         logger.info("feed: " + Util.prettyPrintXml(bundles[0].getDataSets().get(0)));
 
         bundles[0].submitAndScheduleFeed();
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().suspend(bundles[0].getDataSets().get(0)));
         AssertUtil.checkStatus(cluster1OC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
         bundles[1].submitAndScheduleFeed();
         AssertUtil.checkStatus(cluster2OC, EntityType.FEED, bundles[1], Job.Status.RUNNING);


[26/41] git commit: FALCON-338 late data recording is enabled by default for all feeds irrespective of late arrival config. Contributed by Ajay Yadav

Posted by ra...@apache.org.
FALCON-338 late data recording is enabled by default for all feeds irrespective of late arrival config. Contributed by Ajay Yadav


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

Branch: refs/heads/FALCON-585
Commit: 6f770a34df400f3399300ea371a46c1ae0878b06
Parents: d9c115e
Author: Suhas V <su...@inmobi.com>
Authored: Fri Sep 5 00:24:18 2014 +0530
Committer: Suhas V <su...@inmobi.com>
Committed: Fri Sep 5 00:24:18 2014 +0530

----------------------------------------------------------------------
 CHANGES.txt                                           |  3 +++
 .../java/org/apache/falcon/entity/EntityUtil.java     |  5 +++++
 .../java/org/apache/falcon/entity/EntityUtilTest.java | 14 ++++++++++++++
 .../falcon/rerun/handler/TestLateRerunHandler.java    | 13 +++++++++++++
 4 files changed, 35 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6f770a34/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2086f5b..3e9dcf6 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -77,6 +77,9 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-338 - late data recording is enabled by default for all feeds 
+   irrespective of late arrival config (Ajay Yadav via Suhas Vasu)
+
    FALCON-652 EntityUtils tests are failing (Ajay Yadav via Venkatesh Seetharam)
 
    FALCON-650 Instance list APIs occassionally fail when orderBy set to

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6f770a34/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
index 8f258fb..9b66aed 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -613,6 +613,11 @@ public final class EntityUtil {
                 return null;
             }
 
+            //If late Arrival is not configured do not process further
+            if(((Feed) entity).getLateArrival() == null){
+                return null;
+            }
+
             LateProcess lateProcess = new LateProcess();
             lateProcess.setDelay(new Frequency(RuntimeProperties.get().getProperty("feed.late.frequency", "hours(3)")));
             lateProcess.setPolicy(

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6f770a34/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
index 50ee649..3d501a7 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
@@ -19,10 +19,12 @@
 package org.apache.falcon.entity;
 
 import org.apache.falcon.Pair;
+import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.entity.v0.feed.Feed;
+import org.apache.falcon.entity.v0.feed.LateArrival;
 import org.apache.falcon.entity.v0.process.Cluster;
 import org.apache.falcon.entity.v0.process.Process;
 import org.testng.Assert;
@@ -231,4 +233,16 @@ public class EntityUtilTest extends AbstractTestBase {
         Assert.assertEquals(startEndDates.first, expectedStartDate);
         Assert.assertEquals(startEndDates.second, expectedEndDate);
     }
+
+    @Test
+    public void testGetLateProcessFeed() throws FalconException{
+        Feed feed = new Feed();
+
+        Assert.assertNull(EntityUtil.getLateProcess(feed));
+        LateArrival lateArrival = new LateArrival();
+        lateArrival.setCutOff(Frequency.fromString("days(1)"));
+        feed.setLateArrival(lateArrival);
+        Assert.assertNotNull(EntityUtil.getLateProcess(feed));
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6f770a34/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateRerunHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateRerunHandler.java b/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateRerunHandler.java
index 8137f60..93fac13 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateRerunHandler.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateRerunHandler.java
@@ -47,4 +47,17 @@ public class TestLateRerunHandler {
         cutOff = LateRerunHandler.getCutOffTime(feed, nm);
         Assert.assertTrue(cutOff.after(new Date()));
     }
+
+    /**
+     * This test checks that LateData Handler is invoked only if LateArrival
+     * is configured for a feed.
+     */
+    @Test
+    public void testFeedLateArrivalCheck() throws FalconException {
+        Feed feed = new Feed();
+        String nominalTime = "2013-10-01T12:00Z";
+        Date cutOff = LateRerunHandler.getCutOffTime(feed, nominalTime);
+        Assert.assertEquals(cutOff, new Date(0));
+
+    }
 }


[09/41] git commit: FALCON-572 HadoopUtil cleanup in falcon-regression contributed by Ruslan Ostafiychuk

Posted by ra...@apache.org.
FALCON-572 HadoopUtil cleanup in falcon-regression contributed by Ruslan Ostafiychuk


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

Branch: refs/heads/FALCON-585
Commit: e21ec9367b1a4c8b0a359acee6c35c44569180fb
Parents: b1c13df
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Thu Aug 28 16:18:43 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Thu Aug 28 16:18:43 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  1 +
 .../falcon/regression/core/util/HadoopUtil.java | 66 ++------------------
 .../falcon/regression/prism/RetentionTest.java  |  9 ++-
 3 files changed, 15 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e21ec936/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 52e697e..ad6ef79 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -10,6 +10,7 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
    
+   FALCON-572 HadoopUtil cleanup in falcon-regression (Ruslan Ostafiychuk via Samarth Gupta)  
    FALCON-632 Refactoring, documentation stuff (Paul Isaychuk via Samarth Gupta)
 
    FALCON-609 UpdateAtSpecificTimeTest, InstanceSummaryTest tagged, fixed, refactored

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e21ec936/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
index dc0a4d4..af5c01a 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/HadoopUtil.java
@@ -260,31 +260,6 @@ public final class HadoopUtil {
         return locations;
     }
 
-    public static void createLateDataFoldersWithRandom(FileSystem fs, String folderPrefix,
-        List<String> folderList) throws IOException {
-        LOGGER.info("creating late data folders.....");
-        folderList.add(SOMETHING_RANDOM);
-
-        for (final String folder : folderList) {
-            fs.mkdirs(new Path(folderPrefix + folder));
-        }
-
-        LOGGER.info("created all late data folders.....");
-    }
-
-    public static void copyDataToFolders(FileSystem fs, List<String> folderList,
-        String directory, String folderPrefix) throws IOException {
-        LOGGER.info("copying data into folders....");
-        List<String> fileLocations = new ArrayList<String>();
-        File[] files = new File(directory).listFiles();
-        if (files != null) {
-            for (final File file : files) {
-                fileLocations.add(file.toString());
-            }
-        }
-        copyDataToFolders(fs, folderPrefix, folderList,
-                fileLocations.toArray(new String[fileLocations.size()]));
-    }
 
     public static void copyDataToFolders(FileSystem fs, final String folderPrefix,
         List<String> folderList, String... fileLocations) throws IOException {
@@ -321,12 +296,11 @@ public final class HadoopUtil {
     public static void lateDataReplenish(FileSystem fs, int interval,
         int minuteSkip, String folderPrefix) throws IOException {
         List<String> folderData = TimeUtil.getMinuteDatesOnEitherSide(interval, minuteSkip);
-
-        createLateDataFoldersWithRandom(fs, folderPrefix, folderData);
-        copyDataToFolders(fs, folderData, OSUtil.NORMAL_INPUT, folderPrefix);
+        folderData.add(SOMETHING_RANDOM);
+        flattenAndPutDataInFolder(fs, OSUtil.NORMAL_INPUT, folderPrefix, folderData);
     }
 
-    public static void createLateDataFolders(FileSystem fs, final String folderPrefix,
+    public static void createFolders(FileSystem fs, final String folderPrefix,
                                              List<String> folderList) throws IOException {
         for (final String folder : folderList) {
             fs.mkdirs(new Path(folderPrefix + folder));
@@ -353,7 +327,7 @@ public final class HadoopUtil {
         List<String> folderPaths = TimeUtil.getMinuteDatesOnEitherSide(interval, minuteSkip);
         LOGGER.info("folderData: " + folderPaths.toString());
 
-        createLateDataFolders(fs, folderPrefix, folderPaths);
+        createFolders(fs, folderPrefix, folderPaths);
 
         if (fileToBePut.equals("_SUCCESS")) {
             copyDataToFolders(fs, folderPrefix, folderPaths, OSUtil.NORMAL_INPUT + "_SUCCESS");
@@ -374,7 +348,7 @@ public final class HadoopUtil {
             }
         }
 
-        createLateDataFolders(fs, folderPrefix, folderPaths);
+        createFolders(fs, folderPrefix, folderPaths);
         copyDataToFolders(fs, folderPrefix, folderPaths,
                 OSUtil.NORMAL_INPUT + "log_01.txt");
     }
@@ -390,36 +364,8 @@ public final class HadoopUtil {
             }
         }
 
-        createLateDataFolders(fs, folderPrefix, folderPaths);
+        createFolders(fs, folderPrefix, folderPaths);
         copyDataToFolders(fs, folderPrefix, folderPaths,
             OSUtil.NORMAL_INPUT + "_SUCCESS", OSUtil.NORMAL_INPUT + "log_01.txt");
     }
-
-    /**
-     * Removes general folder on file system. Creates folders according to passed folders list
-     * and fills them with data if required.
-     *
-     * @param fileSystem destination file system
-     * @param prefix prefix of path where data should be placed
-     * @param folderList list of folders to be created and filled with data if required
-     * @param uploadData if folders should be filled with data
-     * @throws IOException
-     */
-    public static void replenishData(FileSystem fileSystem, String prefix, List<String> folderList,
-        boolean uploadData) throws IOException {
-        //purge data first
-        deleteDirIfExists(prefix, fileSystem);
-
-        folderList.add(SOMETHING_RANDOM);
-
-        for (final String folder : folderList) {
-            final String pathString = prefix + folder;
-            LOGGER.info(fileSystem.getUri() + pathString);
-            fileSystem.mkdirs(new Path(pathString));
-            if (uploadData) {
-                fileSystem.copyFromLocalFile(new Path(OSUtil.RESOURCES + "log_01.txt"),
-                        new Path(pathString));
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e21ec936/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
index 1d900d9..991002a 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
@@ -30,6 +30,7 @@ import org.apache.falcon.regression.core.supportClasses.JmsMessageConsumer;
 import org.apache.falcon.regression.core.util.AssertUtil;
 import org.apache.falcon.regression.core.util.BundleUtil;
 import org.apache.falcon.regression.core.util.HadoopUtil;
+import org.apache.falcon.regression.core.util.OSUtil;
 import org.apache.falcon.regression.core.util.MathUtil;
 import org.apache.falcon.regression.core.util.OozieUtil;
 import org.apache.falcon.regression.core.util.TimeUtil;
@@ -143,7 +144,13 @@ public class RetentionTest extends BaseTestClass {
             feedType.addTime(today, -36), feedType.addTime(today, 36), skip, feedType);
         final List<String> dataDates = TimeUtil.convertDatesToString(times, feedType.getFormatter());
         logger.info("dataDates = " + dataDates);
-        HadoopUtil.replenishData(clusterFS, testHDFSDir, dataDates, withData);
+        dataDates.add(HadoopUtil.SOMETHING_RANDOM);
+        if (withData) {
+            HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.RESOURCES + "log_01.txt",
+                testHDFSDir, dataDates);
+        } else {
+            HadoopUtil.createFolders(clusterFS, testHDFSDir, dataDates);
+        }
     }
 
     /**


[32/41] git commit: FALCON-338 late data recording is enabled by default for all feeds irrespective of late arrival config(2). Fixing checkstyle bug

Posted by ra...@apache.org.
FALCON-338 late data recording is enabled by default for all feeds irrespective of late arrival config(2). Fixing checkstyle bug


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

Branch: refs/heads/FALCON-585
Commit: 581e5923e7589d9625fc5d49cad13f1e46569706
Parents: 1a3728b
Author: Suhas V <su...@inmobi.com>
Authored: Fri Sep 5 23:11:20 2014 +0530
Committer: Suhas V <su...@inmobi.com>
Committed: Fri Sep 5 23:11:20 2014 +0530

----------------------------------------------------------------------
 common/src/main/java/org/apache/falcon/entity/EntityUtil.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/581e5923/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
index 9b66aed..b8f2d7d 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -614,7 +614,7 @@ public final class EntityUtil {
             }
 
             //If late Arrival is not configured do not process further
-            if(((Feed) entity).getLateArrival() == null){
+            if (((Feed) entity).getLateArrival() == null){
                 return null;
             }
 


[21/41] git commit: FALCON-653 Add falcon regression test for zero input process contributed by Karishma

Posted by ra...@apache.org.
FALCON-653 Add falcon regression test for zero input process contributed by Karishma


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

Branch: refs/heads/FALCON-585
Commit: 75f06b4fe82db9f24aca1036a45c7ad0e26d8f1e
Parents: 2ed0112
Author: Samarth Gupta <sa...@inmobi.com>
Authored: Wed Sep 3 11:31:21 2014 +0530
Committer: Samarth Gupta <sa...@inmobi.com>
Committed: Wed Sep 3 11:31:21 2014 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |  2 +-
 .../prism/PrismProcessScheduleTest.java         | 52 ++++++++++++++++++--
 2 files changed, 48 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/75f06b4f/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 72daf8e..94403c8 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -9,7 +9,7 @@ Trunk (Unreleased)
    via Samarth Gupta)
 
   IMPROVEMENTS
-
+   FALCON-653 Add falcon regression test for zero input process(Karishma via Samarth Gupta)
    FALCON-655 Skip workflow upload if process won't be submitted (Ruslan Ostafiychuk)
 
    FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/75f06b4f/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
index 7822729..10df6c2 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
@@ -33,6 +33,7 @@ import org.apache.falcon.regression.core.util.Util;
 import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
+import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.TestNGException;
@@ -53,11 +54,13 @@ public class PrismProcessScheduleTest extends BaseTestClass {
     OozieClient cluster1OC = serverOC.get(0);
     OozieClient cluster2OC = serverOC.get(1);
     String aggregateWorkflowDir = baseHDFSDir + "/PrismProcessScheduleTest/aggregator";
+    String workflowForNoIpOp = baseHDFSDir + "/PrismProcessScheduleTest/noinop";
     private static final Logger logger = Logger.getLogger(PrismProcessScheduleTest.class);
 
     @BeforeClass(alwaysRun = true)
     public void uploadWorkflow() throws Exception {
         uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
+        uploadDirToClusters(workflowForNoIpOp, OSUtil.RESOURCES+"workflows/aggregatorNoOutput/");
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -388,10 +391,49 @@ public class PrismProcessScheduleTest extends BaseTestClass {
                 hadoopFileEditor.restore();
             }
         }
-    }
+     }
 
-    @AfterClass(alwaysRun = true)
-    public void tearDownClass() throws IOException {
-        cleanTestDirs();
-    }
+     /**
+     * Schedule a process that contains no inputs. The process should be successfully scheduled.
+     *
+     * @throws Exception
+     */
+     @Test(groups = {"prism", "0.2", "embedded"}, enabled = true, timeOut = 1800000)
+     public void testScheduleWhenZeroInputs()throws Exception{
+         bundles[0].submitClusters(prism);
+         bundles[0].setProcessWorkflow(workflowForNoIpOp);
+         ProcessMerlin processObj = new ProcessMerlin(bundles[0].getProcessData());
+         processObj.setInputs(null);
+         processObj.setLateProcess(null);
+         AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(Util.URLS.SUBMIT_URL, bundles[0].getOutputFeedFromBundle()));
+         AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
+                 processObj.toString()));
+         InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(processObj.toString()), 2,
+                 CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS, 10);
+     }
+
+     /**
+     * Schedule a process that contains no inputs or outputs. The process should be successfully scheduled.
+     *
+     * @throws Exception
+     */
+     @Test(groups = {"prism", "0.2", "embedded"}, enabled = true, timeOut = 1800000)
+     public void testScheduleWhenZeroInputsZeroOutputs()throws Exception{
+         bundles[0].submitClusters(prism);
+         bundles[0].setProcessWorkflow(workflowForNoIpOp);
+         ProcessMerlin processObj = new ProcessMerlin(bundles[0].getProcessData());
+         processObj.setInputs(null);
+         processObj.setOutputs(null);
+         processObj.setLateProcess(null);
+         AssertUtil.assertSucceeded(prism.getProcessHelper().submitAndSchedule(Util.URLS.SUBMIT_AND_SCHEDULE_URL,
+                 processObj.toString()));
+         InstanceUtil.waitTillInstanceReachState(cluster1OC, Util.readEntityName(processObj.toString()), 2,
+                 CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS, 10);
+     }
+
+
+     @AfterClass(alwaysRun = true)
+     public void tearDownClass() throws IOException {
+         cleanTestDirs();
+     }
 }


[19/41] git commit: FALCON-650 Instance list APIs occassionally fail when orderBy set to starttime or endtime. Contributed by Balu Vellanki

Posted by ra...@apache.org.
FALCON-650 Instance list APIs occassionally fail when orderBy set to starttime or endtime. Contributed by Balu Vellanki


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

Branch: refs/heads/FALCON-585
Commit: 7cfa00dbdffff166e37b9fc15c633356d1b30f8f
Parents: 8f30ae0
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Tue Sep 2 13:11:41 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Tue Sep 2 13:11:41 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                                  | 3 +++
 .../org/apache/falcon/resource/AbstractInstanceManager.java  | 8 ++++++--
 webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java  | 5 +++++
 3 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/7cfa00db/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 45f0ac3..085fa8e 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -72,6 +72,9 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
+   FALCON-650 Instance list APIs occassionally fail when orderBy set to
+   starttime or endtime (Balu Vellanki via Venkatesh Seetharam)
+
    FALCON-649 Remove unnecessary validation for Instance start time in
    FalconCLI (Balu Vellanki via Venkatesh Seetharam)
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/7cfa00db/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
index 1ffe471..e2c465a 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
@@ -281,14 +281,18 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
             Collections.sort(instanceSet, new Comparator<Instance>() {
                 @Override
                 public int compare(Instance i1, Instance i2) {
-                    return i2.getStartTime().compareTo(i1.getStartTime()); //default desc
+                    Date start1 = (i1.getStartTime() == null) ? new Date(0) : i1.getStartTime();
+                    Date start2 = (i2.getStartTime() == null) ? new Date(0) : i2.getStartTime();
+                    return start2.compareTo(start1); //default desc
                 }
             });
         } else if (orderBy.equals("endTime")) {
             Collections.sort(instanceSet, new Comparator<Instance>() {
                 @Override
                 public int compare(Instance i1, Instance i2) {
-                    return i2.getEndTime().compareTo(i1.getEndTime()); //default desc
+                    Date end1 = (i1.getEndTime() == null) ? new Date(0) : i1.getEndTime();
+                    Date end2 = (i2.getEndTime() == null) ? new Date(0) : i2.getEndTime();
+                    return end2.compareTo(end1); //default desc
                 }
             });
         }//Default : no sort

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/7cfa00db/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
index df99c23..1612abb 100644
--- a/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
+++ b/webapp/src/test/java/org/apache/falcon/cli/FalconCLIIT.java
@@ -813,6 +813,11 @@ public class FalconCLIIT {
                 executeWithURL("instance -logs -type process -name "
                         + overlay.get("processName")
                         + " -start " + START_INSTANCE + " -end " + START_INSTANCE
+                        + " -filterBy STATUS:WAITING -orderBy startTime -offset 0 -numResults 1"));
+        Assert.assertEquals(0,
+                executeWithURL("instance -logs -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE
                         + " -filterBy STATUS:SUCCEEDED,STARTEDAFTER:"+START_INSTANCE+" -offset 1 -numResults 1"));
         Assert.assertEquals(-1,
                 executeWithURL("instance -logs -type process -name "


[27/41] git commit: FALCON-644 Falcon message producer masks errors in Post processing. Contributed by Venkatesh Seetharam

Posted by ra...@apache.org.
FALCON-644 Falcon message producer masks errors in Post processing. Contributed by Venkatesh Seetharam


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

Branch: refs/heads/FALCON-585
Commit: 331ad145e172c4a862bb294578dfb333b39d28d6
Parents: 6f770a3
Author: Venkatesh Seetharam <ve...@apache.org>
Authored: Thu Sep 4 14:40:03 2014 -0700
Committer: Venkatesh Seetharam <ve...@apache.org>
Committed: Thu Sep 4 14:40:03 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  5 ++-
 .../falcon/messaging/JMSMessageConsumer.java    | 36 ++++++----------
 .../falcon/messaging/JMSMessageProducer.java    | 12 ++----
 .../messaging/JMSMessageProducerTest.java       | 17 ++++++++
 .../org/apache/falcon/aspect/GenericAlert.java  |  7 ++++
 .../falcon/workflow/FalconPostProcessing.java   |  7 +++-
 .../workflow/FalconPostProcessingTest.java      | 44 ++++++++++++++++++++
 7 files changed, 95 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3e9dcf6..1abbb67 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -77,7 +77,10 @@ Trunk (Unreleased)
   OPTIMIZATIONS
 
   BUG FIXES
-   FALCON-338 - late data recording is enabled by default for all feeds 
+   FALCON-644 Falcon message producer masks errors in Post processing
+   (Venkatesh Seetharam)
+
+   FALCON-338 - late data recording is enabled by default for all feeds
    irrespective of late arrival config (Ajay Yadav via Suhas Vasu)
 
    FALCON-652 EntityUtils tests are failing (Ajay Yadav via Venkatesh Seetharam)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
index 573f5bd..ec7bd93 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
@@ -20,10 +20,12 @@ package org.apache.falcon.messaging;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
+import org.apache.falcon.aspect.GenericAlert;
 import org.apache.falcon.workflow.WorkflowExecutionArgs;
 import org.apache.falcon.workflow.WorkflowExecutionContext;
 import org.apache.falcon.workflow.WorkflowJobEndNotificationService;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.jms.Connection;
 import javax.jms.ConnectionFactory;
@@ -44,19 +46,18 @@ import java.util.Map;
  * Subscribes to the falcon topic for handling retries and alerts.
  */
 public class JMSMessageConsumer implements MessageListener, ExceptionListener {
-    private static final Logger LOG = Logger.getLogger(JMSMessageConsumer.class);
+    private static final Logger LOG = LoggerFactory.getLogger(JMSMessageConsumer.class);
 
     private final String implementation;
     private final String userName;
     private final String password;
     private final String url;
     private final String topicName;
+    private final WorkflowJobEndNotificationService jobEndNotificationService;
 
     private Connection connection;
     private TopicSubscriber subscriber;
 
-    private final WorkflowJobEndNotificationService jobEndNotificationService;
-
     public JMSMessageConsumer(String implementation, String userName,
                               String password, String url, String topicName,
                               WorkflowJobEndNotificationService jobEndNotificationService) {
@@ -87,22 +88,20 @@ public class JMSMessageConsumer implements MessageListener, ExceptionListener {
     @Override
     public void onMessage(Message message) {
         MapMessage mapMessage = (MapMessage) message;
+        LOG.info("Received message {}", message.toString());
 
         try {
-            if (LOG.isDebugEnabled()) {debug(mapMessage); }
-
             WorkflowExecutionContext context = createContext(mapMessage);
             if (context.hasWorkflowFailed()) {
                 onFailure(context);
             } else if (context.hasWorkflowSucceeded()) {
                 onSuccess(context);
             }
-        } catch (JMSException e) {
-            LOG.info("Error in onMessage for subscriber of topic: " + this.toString(), e);
-        } catch (FalconException e) {
-            LOG.info("Error in onMessage for subscriber of topic: " + this.toString(), e);
         } catch (Exception e) {
-            LOG.info("Error in onMessage for subscriber of topic: " + this.toString(), e);
+            String errorMessage = "Error in onMessage for subscriber of topic: "
+                    + topicName + ", Message: " + message.toString();
+            LOG.info(errorMessage, e);
+            GenericAlert.alertJMSMessageConsumerFailed(errorMessage, e);
         }
     }
 
@@ -127,20 +126,11 @@ public class JMSMessageConsumer implements MessageListener, ExceptionListener {
         jobEndNotificationService.notifySuccess(context);
     }
 
-    private void debug(MapMessage mapMessage) throws JMSException {
-        StringBuilder buff = new StringBuilder();
-        buff.append("Received:{");
-        for (WorkflowExecutionArgs arg : WorkflowExecutionArgs.values()) {
-            buff.append(arg.getName()).append('=')
-                .append(mapMessage.getString(arg.getName())).append(", ");
-        }
-        buff.append("}");
-        LOG.debug(buff);
-    }
-
     @Override
     public void onException(JMSException ignore) {
-        LOG.info("Error in onException for subscriber of topic: " + this.toString(), ignore);
+        String errorMessage = "Error in onException for subscriber of topic: " + topicName;
+        LOG.info(errorMessage, ignore);
+        GenericAlert.alertJMSMessageConsumerFailed(errorMessage, ignore);
     }
 
     public void closeSubscriber() throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
index fc31bab..0181e74 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageProducer.java
@@ -143,9 +143,9 @@ public class JMSMessageProducer {
      * Sends all arguments.
      *
      * @return error code
-     * @throws JMSException
+     * @throws Exception
      */
-    public int sendMessage() throws JMSException {
+    public int sendMessage() throws Exception {
         return sendMessage(WorkflowExecutionArgs.values());
     }
 
@@ -156,9 +156,9 @@ public class JMSMessageProducer {
      *
      * @param filteredArgs args sent in the message.
      * @return error code
-     * @throws JMSException
+     * @throws Exception
      */
-    public int sendMessage(WorkflowExecutionArgs[] filteredArgs) throws JMSException {
+    public int sendMessage(WorkflowExecutionArgs[] filteredArgs) throws Exception {
         List<Map<String, String>> messageList = buildMessageList(filteredArgs);
 
         if (messageList.isEmpty()) {
@@ -174,10 +174,6 @@ public class JMSMessageProducer {
                 LOG.info("Sending message: {}", message);
                 sendMessage(connection, message);
             }
-        } catch (JMSException e) {
-            LOG.error("Error in getConnection:", e);
-        } catch (Exception e) {
-            LOG.error("Error in getConnection:", e);
         } finally {
             closeQuietly(connection);
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
index e4ea22f..bf8615f 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageProducerTest.java
@@ -215,4 +215,21 @@ public class JMSMessageProducerTest {
                 "2012-01-01-01-00");
         Assert.assertEquals(message.getString(WorkflowExecutionArgs.STATUS.getName()), "SUCCEEDED");
     }
+
+    @Test (expectedExceptions = JMSException.class)
+    public void testFailuresInSendMessagesAreNotMasked() throws Exception {
+        List<String> args = new ArrayList<String>(Arrays.asList(
+                "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), "agg-coord",
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "raw-logs",
+                "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
+                "/raw-logs/10/05/05/00/20",
+                "-" + WorkflowExecutionArgs.BRKR_URL.getName(), "error"));
+        args.addAll(createCommonArgs());
+
+        WorkflowExecutionContext context = WorkflowExecutionContext.create(
+                args.toArray(new String[args.size()]), WorkflowExecutionContext.Type.POST_PROCESSING);
+        JMSMessageProducer jmsMessageProducer = JMSMessageProducer.builder(context)
+                .type(JMSMessageProducer.MessageType.FALCON).build();
+        jmsMessageProducer.sendMessage();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
index 5ab2f72..d2019b5 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
@@ -26,6 +26,7 @@ import org.aspectj.lang.annotation.Aspect;
  * Create a method with params you want to monitor via Aspect and log in metric
  * and iMon, invoke this method from code.
  */
+@SuppressWarnings("UnusedParameters")
 @Aspect
 public final class GenericAlert {
 
@@ -107,6 +108,12 @@ public final class GenericAlert {
             @Dimension(value = "message") String message,
             @Dimension(value = "exception") Throwable throwable) {
         return "IGNORE";
+    }
 
+    @Monitored(event = "jms-message-consumer-failed")
+    public static String alertJMSMessageConsumerFailed(
+            @Dimension(value = "error-message") String errorMessage,
+            @Dimension(value = "exception") Throwable throwable) {
+        return "IGNORE";
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
index 0adc11b..e5b3704 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
@@ -80,6 +80,11 @@ public class FalconPostProcessing extends Configured implements Tool {
             return;
         }
 
-        new JobLogMover().run(context);
+        try {
+            new JobLogMover().run(context);
+        } catch (Exception ignored) {
+            // Mask exception, a failed log mover will not fail the user workflow
+            LOG.error("Exception in job log mover:", ignored);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/331ad145/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
index 201b682..87d5e8a 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
@@ -154,4 +154,48 @@ public class FalconPostProcessingTest {
         Assert.assertEquals(m.getString(WorkflowExecutionArgs.TIMESTAMP.getName()), "2012-01-01-01-00");
         Assert.assertEquals(m.getString(WorkflowExecutionArgs.STATUS.getName()), "SUCCEEDED");
     }
+
+    @Test (expectedExceptions = JMSException.class)
+    public void testFailuresInSendMessagesAreNotMasked() throws Exception {
+        try {
+            broker.stop();
+        } catch (Exception ignored) {
+            // ignore
+        } finally {
+            new FalconPostProcessing().run(getMessageArgs());
+        }
+    }
+
+    private String[] getMessageArgs() {
+        return new String[]{
+            "-" + WorkflowExecutionArgs.ENTITY_NAME.getName(), ENTITY_NAME,
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_NAMES.getName(), "out-click-logs,out-raw-logs",
+            "-" + WorkflowExecutionArgs.OUTPUT_FEED_PATHS.getName(),
+            "/out-click-logs/10/05/05/00/20,/out-raw-logs/10/05/05/00/20",
+            "-" + WorkflowExecutionArgs.WORKFLOW_ID.getName(), "workflow-01-00",
+            "-" + WorkflowExecutionArgs.WORKFLOW_USER.getName(), "falcon",
+            "-" + WorkflowExecutionArgs.RUN_ID.getName(), "1",
+            "-" + WorkflowExecutionArgs.NOMINAL_TIME.getName(), "2011-01-01-01-00",
+            "-" + WorkflowExecutionArgs.TIMESTAMP.getName(), "2012-01-01-01-00",
+            "-" + WorkflowExecutionArgs.BRKR_URL.getName(), "error",
+            "-" + WorkflowExecutionArgs.BRKR_IMPL_CLASS.getName(), BROKER_IMPL_CLASS,
+            "-" + WorkflowExecutionArgs.USER_BRKR_URL.getName(), "error",
+            "-" + WorkflowExecutionArgs.USER_BRKR_IMPL_CLASS.getName(), BROKER_IMPL_CLASS,
+            "-" + WorkflowExecutionArgs.ENTITY_TYPE.getName(), "process",
+            "-" + WorkflowExecutionArgs.OPERATION.getName(), "GENERATE",
+            "-" + WorkflowExecutionArgs.LOG_FILE.getName(), "/logFile",
+            "-" + WorkflowExecutionArgs.STATUS.getName(), "SUCCEEDED",
+            "-" + WorkflowExecutionArgs.BRKR_TTL.getName(), "10",
+            "-" + WorkflowExecutionArgs.CLUSTER_NAME.getName(), "corp",
+            "-" + WorkflowExecutionArgs.WF_ENGINE_URL.getName(), "http://localhost:11000/oozie/",
+            "-" + WorkflowExecutionArgs.LOG_DIR.getName(), "target/log",
+            "-" + WorkflowExecutionArgs.USER_SUBFLOW_ID.getName(), "userflow@wf-id" + "test",
+            "-" + WorkflowExecutionArgs.USER_WORKFLOW_ENGINE.getName(), "oozie",
+            "-" + WorkflowExecutionArgs.INPUT_FEED_NAMES.getName(), "in-click-logs,in-raw-logs",
+            "-" + WorkflowExecutionArgs.INPUT_FEED_PATHS.getName(),
+            "/in-click-logs/10/05/05/00/20,/in-raw-logs/10/05/05/00/20",
+            "-" + WorkflowExecutionArgs.USER_WORKFLOW_NAME.getName(), "test-workflow",
+            "-" + WorkflowExecutionArgs.USER_WORKFLOW_VERSION.getName(), "1.0.0",
+        };
+    }
 }


[13/41] FALCON-587 Don't delete input data in @AfterClass in falcon-regression tests if clean_test_dir=false

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
index 319363c..7822729 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessScheduleTest.java
@@ -36,11 +36,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.TestNGException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 
@@ -355,7 +357,7 @@ public class PrismProcessScheduleTest extends BaseTestClass {
                 .getClusterHelper().getHadoopFS());
 
             hadoopFileEditor.edit(new ProcessMerlin(bundles[0]
-                    .getProcessData()).getWorkflow().getPath() + "/workflow.xml",
+                .getProcessData()).getWorkflow().getPath() + "/workflow.xml",
                 "<value>${outputData}</value>",
                 "<property>\n" +
                     "                    <name>randomProp</name>\n" +
@@ -387,4 +389,9 @@ public class PrismProcessScheduleTest extends BaseTestClass {
             }
         }
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
index efa5b8f..842bc1a 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
@@ -33,11 +33,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 public class PrismProcessSnSTest extends BaseTestClass {
@@ -247,4 +249,8 @@ public class PrismProcessSnSTest extends BaseTestClass {
 
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
index e77ae13..a14cd35 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSuspendTest.java
@@ -31,11 +31,13 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 
 public class PrismProcessSuspendTest extends BaseTestClass {
@@ -300,4 +302,9 @@ public class PrismProcessSuspendTest extends BaseTestClass {
         AssertUtil.assertFailed(cluster2.getProcessHelper()
             .suspend(Util.URLS.SUSPEND_URL, bundles[1].getProcessData()));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
index 8f2b160..cdd02da 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismSubmitTest.java
@@ -30,11 +30,13 @@ import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.net.ConnectException;
 import java.util.List;
@@ -589,4 +591,8 @@ public class PrismSubmitTest extends BaseTestClass {
         AssertUtil.compareDataStoreStates(beforeSubmitCluster2, afterSubmitCluster2, 0);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/ProcessPartitionExpVariableTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/ProcessPartitionExpVariableTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/ProcessPartitionExpVariableTest.java
index 5c3cf1a..1a3f82f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/ProcessPartitionExpVariableTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/ProcessPartitionExpVariableTest.java
@@ -35,13 +35,13 @@ import org.apache.oozie.client.OozieClient;
 import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 import org.apache.log4j.Logger;
 
-import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
@@ -150,4 +150,9 @@ public class ProcessPartitionExpVariableTest extends BaseTestClass {
     //TODO: ProcessPartitionExpVariableTest_OptionalPartition()
     //TODO: ProcessPartitionExpVariableTest_CompulsoryPartition()
     //TODO: ProcessPartitionExpVariableTest_moreThanOnceVariable()
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
index 2ba441c..468692a 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleKilledProcessTest.java
@@ -31,11 +31,13 @@ import org.apache.falcon.regression.core.util.Util.URLS;
 import org.apache.falcon.regression.core.util.XmlUtil;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.log4j.Logger;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.List;
@@ -131,4 +133,9 @@ public class RescheduleKilledProcessTest extends BaseTestClass {
         AssertUtil.assertSucceeded(prism.getProcessHelper().schedule(URLS.SCHEDULE_URL,
             bundles[0].getProcessData()));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
index a81475d..7109829 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RescheduleProcessInFinalStatesTest.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.Job.Status;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -206,4 +207,9 @@ public class RescheduleProcessInFinalStatesTest extends BaseTestClass {
         Assert.assertTrue(r.getMessage().contains("(process) not found"));
         AssertUtil.assertFailed(r);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
index 991002a..8bce3de 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/RetentionTest.java
@@ -45,6 +45,7 @@ import org.apache.oozie.client.OozieClientException;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
@@ -295,4 +296,8 @@ public class RetentionTest extends BaseTestClass {
         return MathUtil.crossProduct(retentionPeriods, retentionUnits, gaps, feedTypes, withData);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
index 8bfeda4..f8a9fd6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/UpdateAtSpecificTimeTest.java
@@ -44,6 +44,7 @@ import org.custommonkey.xmlunit.Diff;
 import org.custommonkey.xmlunit.XMLUnit;
 import org.joda.time.DateTime;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -641,4 +642,9 @@ public class UpdateAtSpecificTimeTest extends BaseTestClass {
         bundles[0].deleteBundle(prism);
         processBundle.deleteBundle(prism);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
index c62935e..697d8bd 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/LineageGraphTest.java
@@ -126,6 +126,7 @@ public class LineageGraphTest extends BaseUITestClass {
     public void tearDown() throws IOException {
         closeBrowser();
         removeBundles();
+        cleanTestDirs();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fad72d2a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
index 356920c..458ddc6 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ui/ProcessUITest.java
@@ -46,6 +46,7 @@ import org.apache.log4j.Logger;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.OozieClientException;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -222,4 +223,9 @@ public class ProcessUITest extends BaseUITestClass {
             }
         }
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }