Merge remote-tracking branch master into FALCON-585.
diff --git a/CHANGES.txt b/CHANGES.txt
index 4e3f919..4a3bbc4 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,7 +27,25 @@
    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)
+
+   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)
+
+   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)
+
    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)
 
@@ -59,6 +77,30 @@
   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)
+
+   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
+   starttime or endtime (Balu Vellanki via Venkatesh Seetharam)
+
+   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)
+
+   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
    (Shwetha GS via Suhas Vasu)
 
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..d6e3598 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 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";
@@ -85,8 +86,10 @@
     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";
 
     public static final String INSTANCE_CMD = "instance";
     public static final String START_OPT = "start";
@@ -227,23 +230,27 @@
         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");
+        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";
-        validateInstanceCommands(optionsList, entity, type, start, colo);
+        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)) {
@@ -261,7 +268,7 @@
             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);
@@ -272,7 +279,7 @@
         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 {
@@ -287,7 +294,7 @@
 
     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");
@@ -301,12 +308,6 @@
             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)
@@ -338,14 +339,22 @@
         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 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);
-        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");
+        Integer numInstances = parseIntegerInput(commandLine.getOptionValue(NUM_INSTANCES_OPT), 7, "numInstances");
         validateEntityType(entityType);
+        validateSortOrder(sortOrder);
+        String entityAction = "entity";
 
         if (optionsList.contains(SUBMIT_OPT)) {
             validateFilePath(filePath);
@@ -396,11 +405,18 @@
         } 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);
+                    filterTags, orderBy, sortOrder, 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, sortOrder, offset, numResults, numInstances);
         } else if (optionsList.contains(HELP_CMD)) {
             OUT.get().println("Falcon Help");
         } else {
@@ -409,6 +425,21 @@
         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");
+        }
+    }
+
     private String getColo(String colo) throws FalconCLIException, IOException {
         if (colo == null) {
             Properties prop = getClientProperties();
@@ -563,6 +594,8 @@
                 "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);
@@ -577,6 +610,7 @@
         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,
@@ -586,8 +620,10 @@
                 "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");
@@ -596,10 +632,13 @@
         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,
                 "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);
@@ -607,13 +646,18 @@
         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);
         entityOptions.addOption(filterTags);
         entityOptions.addOption(orderBy);
+        entityOptions.addOption(sortOrder);
         entityOptions.addOption(offset);
         entityOptions.addOption(numResults);
+        entityOptions.addOption(numInstances);
 
         return entityOptions;
     }
@@ -670,7 +714,6 @@
                 false,
                 "Displays the workflow parameters for a given instance of specified nominal time");
 
-
         OptionGroup group = new OptionGroup();
         group.addOption(running);
         group.addOption(list);
@@ -721,6 +764,7 @@
                 "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,
@@ -741,6 +785,7 @@
         instanceOptions.addOption(filterBy);
         instanceOptions.addOption(offset);
         instanceOptions.addOption(orderBy);
+        instanceOptions.addOption(sortOrder);
         instanceOptions.addOption(numResults);
 
         return instanceOptions;
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..73635fa 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.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;
@@ -81,6 +82,8 @@
     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) {
@@ -178,6 +181,7 @@
         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;
@@ -331,28 +335,40 @@
         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 {
+                                    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);
     }
 
-    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
-    public String getRunningInstances(String type, String entity, String colo, List<LifeCycle> lifeCycles,
-                                      String filterBy, String orderBy, Integer offset, Integer numResults)
+    public String getEntitySummary(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 {
+        return sendEntitySummaryRequest(Entities.SUMMARY, entityType, cluster, start, end, fields, filterBy, filterTags,
+                orderBy, sortOrder, offset, numResults, numInstances);
+    }
+
+    public String getRunningInstances(String type, String entity, String colo, List<LifeCycle> lifeCycles,
+                                      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,
@@ -427,11 +443,11 @@
     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,
@@ -443,7 +459,6 @@
         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);
@@ -521,6 +536,79 @@
         return parseAPIResult(clientResponse);
     }
 
+    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(tags)) {
+            resource = resource.queryParam("tags", tags);
+        }
+        if (!StringUtils.isEmpty(filterBy)) {
+            resource = resource.queryParam("filterBy", filterBy);
+        }
+        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 = addParamsToResource(resource, start, end, null, null,
+                fields, filterBy, filterTags,
+                orderBy, sortOrder,
+                offset, numResults, numInstances);
+
+        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 {
 
@@ -571,36 +659,19 @@
                                        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,
                                        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);
@@ -660,24 +731,13 @@
 
     //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)
@@ -722,6 +782,25 @@
         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);
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 @@
      * Filter by these Fields is supported by RestAPI.
      */
     public static enum EntityFilterByFields {
-        TYPE, NAME, STATUS, PIPELINES
+        TYPE, NAME, STATUS, PIPELINES, CLUSTER
     }
 
     /**
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
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..b8f2d7d 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -21,7 +21,9 @@
 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;
 import org.apache.falcon.entity.WorkflowNameBuilder.WorkflowName;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -611,6 +613,11 @@
                 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(
@@ -687,4 +694,65 @@
         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;
+        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);
+    }
+
 }
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..2f9fe8e 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 @@
 
 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.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;
@@ -114,6 +118,12 @@
 
     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 +132,7 @@
             return;
         }
 
-        addEdge(instanceVertex, entityVertex, edgeLabel.getName());
+        addEdge(instanceVertex, entityVertex, edgeLabel.getName(), timestamp);
     }
 
     public void addOutputFeedInstances(WorkflowExecutionContext context,
@@ -166,13 +176,86 @@
         }
     }
 
+    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, context.getNominalTimeAsISO8601());
+        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());
+    }
+
+    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, context.getNominalTimeAsISO8601());
+            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 {
         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());
@@ -193,8 +276,9 @@
         }
     }
 
-    public String getFeedInstanceName(String feedName, String clusterName,
-                                      String feedInstancePath) throws FalconException {
+    public static String getFeedInstanceName(String feedName, String clusterName,
+                                             String feedInstancePath,
+                                             String nominalTime) throws FalconException {
         try {
             Feed feed = ConfigurationStore.get().get(EntityType.FEED, feedName);
             Cluster cluster = ConfigurationStore.get().get(EntityType.CLUSTER, clusterName);
@@ -202,22 +286,23 @@
             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);
         }
     }
 
-    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,
-                                                 Cluster cluster) throws FalconException {
+    private static String getFileSystemFeedInstanceName(String feedInstancePath, Feed feed,
+                                                        Cluster cluster,
+                                                        String nominalTime) throws FalconException {
         Storage rawStorage = FeedHelper.createStorage(cluster, feed);
         String feedPathTemplate = rawStorage.getUriTemplate(LocationType.DATA);
         String instance = feedInstancePath;
@@ -227,7 +312,9 @@
             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);
     }
 }
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..f607e0a 100644
--- a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
+++ b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
@@ -288,13 +288,13 @@
         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) {
+    private void onFeedInstanceEvicted(WorkflowExecutionContext context) throws FalconException {
         LOG.info("Adding evicted feed instance: {}", context.getNominalTimeAsISO8601());
-        // todo - tbd
+        instanceGraphBuilder.addEvictedInstance(context);
     }
 }
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 @@
     }
 
     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) {
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..5b312da 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,13 @@
     CLUSTER_COLO("collocated"),
     USER("owned-by"),
     GROUPS("grouped-as"),
-    PIPELINES("part-of-pipeline");
+    PIPELINES("pipeline"),
+
+    // replication labels
+    FEED_CLUSTER_REPLICATED_EDGE("replicated-to"),
+
+    // eviction labels
+    FEED_CLUSTER_EVICTED_EDGE("evicted-from");
 
     private final String name;
 
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);
+    }
+}
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 @@
 
     // 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),
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..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 static final String OUTPUT_FEED_SEPARATOR = ",";
     public static final String INPUT_FEED_SEPARATOR = "#";
-
+    public static final String CLUSTER_NAME_SEPARATOR = ",";
 
     /**
      * Workflow execution status.
@@ -80,8 +80,8 @@
         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,
@@ -161,7 +161,26 @@
     }
 
     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() {
@@ -177,7 +196,7 @@
     }
 
     public String getOutputFeedNames() {
-        return getValue(WorkflowExecutionArgs.FEED_NAMES);
+        return getValue(WorkflowExecutionArgs.OUTPUT_FEED_NAMES);
     }
 
     public String[] getOutputFeedNamesList() {
@@ -185,7 +204,7 @@
     }
 
     public String getOutputFeedInstancePaths() {
-        return getValue(WorkflowExecutionArgs.FEED_INSTANCE_PATHS);
+        return getValue(WorkflowExecutionArgs.OUTPUT_FEED_PATHS);
     }
 
     public String[] getOutputFeedInstancePathsList() {
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 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()) {
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..3d501a7 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
@@ -18,10 +18,13 @@
 
 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;
@@ -212,4 +215,34 @@
                 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);
+
+        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.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));
+    }
+
 }
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..7b73a91 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.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,12 +40,15 @@
 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;
 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;
@@ -64,24 +68,36 @@
 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 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";
+    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";
     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 =
         "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 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";
 
@@ -89,10 +105,13 @@
     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;
+    private EmbeddedCluster embeddedCluster;
+    private String hdfsUrl;
+    private static String logFilePath;
 
 
     @BeforeClass
@@ -117,9 +136,7 @@
     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 +156,8 @@
 
     @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 +168,35 @@
 
     @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 +207,9 @@
 
     @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 +225,13 @@
         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 +244,103 @@
         Assert.assertEquals(getEdgesCount(service.getGraph()), 71);
     }
 
-    @Test (dependsOnMethods = "testMapLineage")
+    @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();
+
+        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
+    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
         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 +357,7 @@
             // 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 +366,8 @@
         }
 
         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 +388,16 @@
         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 +408,8 @@
         }
 
         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 +449,7 @@
         // 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 +474,40 @@
         }
     }
 
+    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();
@@ -604,11 +721,20 @@
     }
 
     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();
 
@@ -623,29 +749,55 @@
         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 static String[] getTestMessageArgs() {
+    private void verifyLineageGraphForReplicationOrEviction(String feedName, String feedInstanceDataPath,
+                                                            WorkflowExecutionContext context,
+                                                            RelationshipLabel edgeLabel) throws Exception {
+        String feedInstanceName = InstanceRelationshipGraphBuilder.getFeedInstanceName(feedName
+                , context.getSrcClusterName(), feedInstanceDataPath, context.getNominalTimeAsISO8601());
+        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.FEED_NAMES.getName(), OUTPUT_FEED_NAMES,
-            "-" + WorkflowExecutionArgs.FEED_INSTANCE_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 +807,10 @@
 
             "-" + 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,
@@ -667,10 +818,120 @@
             "-" + 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"),
         };
     }
 
+    private void setup() throws Exception {
+        cleanUp();
+        service.init();
+
+        // Add cluster
+        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", cluster,
+                "classified-as=Secure", "analytics", Storage.TYPE.FILESYSTEM,
+                "/falcon/impression-feed/${YEAR}/${MONTH}/${DAY}");
+        inputFeeds.add(impressionsFeed);
+        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", 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", 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 {
+        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 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 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);
+        service.destroy();
+    }
+
     private void cleanupGraphStore(Graph graph) {
         for (Edge edge : graph.getEdges()) {
             graph.removeEdge(edge);
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 @@
             "-" + 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,
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 @@
             "-" + 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,
diff --git a/docs/src/site/twiki/FalconCLI.twiki b/docs/src/site/twiki/FalconCLI.twiki
index 2c5d9c5..3b945e9 100644
--- a/docs/src/site/twiki/FalconCLI.twiki
+++ b/docs/src/site/twiki/FalconCLI.twiki
@@ -51,6 +51,24 @@
 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>> -sortOrder <<sortOrder>> -offset 0 -numResults 10
+
+<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>> -sortOrder <<sortOrder>> -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
@@ -135,7 +153,31 @@
 {"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 -sortOrder <<sortOrder>> -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 -sortOrder <<sortOrder>> -offset 0 -numResults 10
+
+<a href="./Restapi/InstanceList.html">Optional params described here.</a>
 
 ---+++Summary
 
@@ -148,7 +190,12 @@
 {"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 +204,23 @@
 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>> -sortOrder <<sortOrder>> -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 -sortOrder <<sortOrder>> -offset 0 -numResults 10
+
+<a href="./Restapi/InstanceLogs.html">Optional params described here.</a>
 
 ---+++LifeCycle
 
diff --git a/docs/src/site/twiki/restapi/EntityList.twiki b/docs/src/site/twiki/restapi/EntityList.twiki
index 678b907..b569ade 100644
--- a/docs/src/site/twiki/restapi/EntityList.twiki
+++ b/docs/src/site/twiki/restapi/EntityList.twiki
@@ -8,17 +8,19 @@
 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, 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".
+   * 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.
 
 ---++ Results
 List of the entities.
@@ -68,25 +70,25 @@
 
 ---+++ 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"
         }
     ]
 }
diff --git a/docs/src/site/twiki/restapi/EntitySummary.twiki b/docs/src/site/twiki/restapi/EntitySummary.twiki
new file mode 100644
index 0000000..f002d81
--- /dev/null
+++ b/docs/src/site/twiki/restapi/EntitySummary.twiki
@@ -0,0 +1,73 @@
+---++  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".
+   * 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.
+
+---++ 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>
diff --git a/docs/src/site/twiki/restapi/InstanceList.twiki b/docs/src/site/twiki/restapi/InstanceList.twiki
index c2059f2..2cd9b1c 100644
--- a/docs/src/site/twiki/restapi/InstanceList.twiki
+++ b/docs/src/site/twiki/restapi/InstanceList.twiki
@@ -8,19 +8,22 @@
 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".
+   * 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.
    
 ---++ Results
 List of instances of given entity.
diff --git a/docs/src/site/twiki/restapi/InstanceLogs.twiki b/docs/src/site/twiki/restapi/InstanceLogs.twiki
index 56d5fe1..c1103b7 100644
--- a/docs/src/site/twiki/restapi/InstanceLogs.twiki
+++ b/docs/src/site/twiki/restapi/InstanceLogs.twiki
@@ -8,20 +8,24 @@
 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".
+   * 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.
+
 ---++ Results
 Log of specified instance.
 
diff --git a/docs/src/site/twiki/restapi/InstanceRunning.twiki b/docs/src/site/twiki/restapi/InstanceRunning.twiki
index 0d04008..dcd2230 100644
--- a/docs/src/site/twiki/restapi/InstanceRunning.twiki
+++ b/docs/src/site/twiki/restapi/InstanceRunning.twiki
@@ -8,17 +8,19 @@
 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".
+   * 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.
+
 
 ---++ Results
 List of instances currently running.
@@ -48,7 +50,7 @@
 
 ---+++ 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>
diff --git a/docs/src/site/twiki/restapi/InstanceStatus.twiki b/docs/src/site/twiki/restapi/InstanceStatus.twiki
index 5160252..ece8c3f 100644
--- a/docs/src/site/twiki/restapi/InstanceStatus.twiki
+++ b/docs/src/site/twiki/restapi/InstanceStatus.twiki
@@ -8,19 +8,23 @@
 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".
+   * 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.
+
    
 ---++ Results
 Status of the specified instance.
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 @@
 ---+++ 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 @@
                     "entry":
                         {
                             "key":"SUCCEEDED",
-                            "value":"3"
+                            "key2":"value"
                          }
                 }
             }
diff --git a/docs/src/site/twiki/restapi/ResourceList.twiki b/docs/src/site/twiki/restapi/ResourceList.twiki
index 4b377e8..d9cb3cb 100644
--- a/docs/src/site/twiki/restapi/ResourceList.twiki
+++ b/docs/src/site/twiki/restapi/ResourceList.twiki
@@ -48,21 +48,22 @@
 | 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         | [[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
 
 | *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
 
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 2e2b447..2f5de05 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -5,25 +5,62 @@
   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-610 Refactoring and documentation updates (Paul Isaychuk via 
-   Arpit Gupta)
+   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-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) 
@@ -32,5 +69,5 @@
    mismatched (Raghav Kumar Gautam via Arpit Gupta)
 
   OPTIMIZATIONS
-
+  FALCON-681 delete duplicate feed retention test from falcon regression (SamarthG)
   BUG FIXES
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 ff5a6a8..1536d2c 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
@@ -47,7 +47,6 @@
 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;
@@ -81,22 +80,19 @@
     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()
@@ -104,8 +100,7 @@
         submitClusters(prismHelper);
 
         for (String feed : dataSets) {
-            AssertUtil.assertSucceeded(
-                prismHelper.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed));
+            AssertUtil.assertSucceeded(prismHelper.getFeedHelper().submitAndSchedule(feed));
         }
     }
 
@@ -113,8 +108,7 @@
         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);
         }
@@ -129,23 +123,20 @@
 
         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() {
@@ -272,7 +263,7 @@
         //lets submit all data first
         submitFeeds(helper);
 
-        return helper.getProcessHelper().submitEntity(URLS.SUBMIT_URL, getProcessData());
+        return helper.getProcessHelper().submitEntity(getProcessData());
     }
 
     /**
@@ -294,8 +285,7 @@
         }
 
         //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();
@@ -697,16 +687,14 @@
     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));
         }
     }
 
@@ -758,14 +746,14 @@
     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();
             }
@@ -773,7 +761,7 @@
 
         for (String cluster : this.getClusters()) {
             try {
-                helper.getClusterHelper().delete(URLS.DELETE_URL, cluster);
+                helper.getClusterHelper().delete(cluster);
             } catch (Exception e) {
                 e.getStackTrace();
             }
@@ -808,8 +796,7 @@
 
         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());
         }
 
@@ -872,21 +859,18 @@
         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);
         }
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..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,10 @@
     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"));
 
     /* initialize keyTabMap */
     static {
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.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 @@
         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);
     }
 
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 @@
         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 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 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 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 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 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);
     }
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 @@
         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);
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 166fb3d..62838f8 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 @@
         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(String appPath, String testName) throws IOException {
         return generateBundleFromTemplate("updateBundle", appPath, testName);
     }
@@ -93,7 +89,7 @@
         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;
@@ -136,8 +132,7 @@
     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"));
 
         }
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 @@
     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 @@
             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 @@
             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 @@
             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 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));
         }
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 c348348..735a732 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 @@
         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 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 @@
         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 @@
             }
         }
 
-        createLateDataFolders(fs, folderPrefix, folderPaths);
+        createFolders(fs, folderPrefix, folderPaths);
         copyDataToFolders(fs, folderPrefix, folderPaths,
                 OSUtil.NORMAL_INPUT + "log_01.txt");
     }
@@ -390,40 +364,12 @@
             }
         }
 
-        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));
-            }
-        }
-    }
-
-    /**
      * 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
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 @@
             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.");
     }
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..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
@@ -119,12 +119,10 @@
     }
 
     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 @@
                 temp.setStatus(APIResult.Status.FAILED);
             }
         }
-
         return temp;
     }
 
@@ -170,30 +167,25 @@
     }
 
     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 @@
                 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 @@
         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 @@
         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 @@
                 .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 @@
         for (String line : raw) {
             finalList.add(line.split(",")[0]);
         }
-
         return finalList;
     }
 
@@ -327,7 +309,6 @@
 
     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 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 @@
                 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 @@
         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 @@
     }
 
     /**
-     * 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 @@
         }
     }
 
-
+    /**
+     * @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 @@
                 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 @@
         } 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 @@
         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,7 @@
         } else {
             helper = cluster.getClusterHelper();
         }
-
-        ServiceResponse response = helper.getEntityDefinition(URLS
-            .GET_ENTITY_DEFINITION, entity);
-
+        ServiceResponse response = helper.getEntityDefinition(entity);
         if (shouldReturn) {
             AssertUtil.assertSucceeded(response);
         } else {
@@ -608,7 +616,6 @@
         }
         String result = response.getMessage();
         Assert.assertNotNull(result);
-
         return result;
     }
 }
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 d892332..bc78699 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
@@ -55,6 +55,7 @@
         "/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() {
@@ -114,4 +115,12 @@
             }
         }
     }
+
+    public void cleanTestDirs() throws IOException {
+        if (MerlinConstants.CLEAN_TEST_DIR) {
+            for (FileSystem fs : serverFS) {
+                HadoopUtil.deleteDirIfExists(baseHDFSDir, fs);
+            }
+        }
+    }
 }
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 eb41808..85944ef 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.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;
@@ -69,8 +70,7 @@
     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 {
@@ -96,7 +96,7 @@
         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 @@
         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 @@
         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 @@
             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 @@
         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 @@
         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 @@
         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 @@
         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 @@
             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 @@
         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 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,16 +522,14 @@
         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");
         //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,
@@ -555,12 +546,11 @@
         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/${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,
@@ -579,8 +569,7 @@
         //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 @@
         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);
 
@@ -650,8 +638,8 @@
             .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);
@@ -669,6 +657,34 @@
         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(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(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,
@@ -690,4 +706,9 @@
         KerberosHelper.loginFromKeytab(MerlinConstants.CURRENT_USER_NAME);
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 d19daed..24e94ea 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.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -50,7 +51,6 @@
     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";
@@ -69,14 +69,13 @@
         String startDate = TimeUtil.getTimeWrtSystemTime(-20);
         String endDate = TimeUtil.getTimeWrtSystemTime(70);
 
-        b.setInputFeedDataPath(testInputDir + "/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        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);
 
-        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT, prefix, dataDates);
+        HadoopUtil.flattenAndPutDataInFolder(clusterFS, OSUtil.NORMAL_INPUT,
+            b.getFeedDataPathPrefix(), dataDates);
     }
 
     @BeforeMethod(alwaysRun = true)
@@ -85,10 +84,8 @@
         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(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);
@@ -121,8 +118,7 @@
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        logger.info("in @AfterClass");
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 5917a42..b5d1c62 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.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 @@
         return df.format(num);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 6a9edd3..85fed2f 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.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;
@@ -50,6 +49,7 @@
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -62,36 +62,30 @@
     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";
-    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;
+    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(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)
@@ -101,8 +95,7 @@
         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");
@@ -120,6 +113,8 @@
         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,76 +124,56 @@
 
     @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(process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        ServiceResponse status =
-            prism.getProcessHelper().getStatus(URLS.STATUS_URL, bundles[0].getProcessData());
+        ServiceResponse status = prism.getProcessHelper().getStatus(process);
         Assert.assertTrue(status.getMessage().contains("SUSPENDED"), "Process not suspended.");
-        prism.getProcessHelper().resume(URLS.RESUME_URL, bundles[0].getProcessData());
+        prism.getProcessHelper().resume(process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        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, bundles[0].getProcessData());
+        prism.getProcessHelper().suspend(process);
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.SUSPENDED);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
     @Test(groups = {"singleCluster"})
     public void getRunningProcessInstance() throws Exception {
-        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, bundles[0].getProcessData(),
-            Job.Status.RUNNING);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        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, bundles[0].getProcessData());
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        prism.getProcessHelper().delete(process);
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(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);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.RUNNING);
+        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,
-                Util.readEntityName(bundles[0].getProcessData()));
+        r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        logger.info("in @AfterClass");
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 e09c2fc..5e7a3bb 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,17 +30,18 @@
 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.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;
 
 /**
@@ -124,13 +125,12 @@
 
         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);
 
@@ -182,8 +182,7 @@
         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);
@@ -222,13 +221,12 @@
 
         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);
 
@@ -282,8 +280,7 @@
         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),
@@ -317,13 +314,12 @@
 
         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);
 
@@ -377,8 +373,7 @@
         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),
@@ -412,14 +407,13 @@
 
         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);
 
@@ -471,8 +465,7 @@
         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),
@@ -506,13 +499,12 @@
 
         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);
 
@@ -566,8 +558,7 @@
         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),
@@ -612,14 +603,13 @@
 
         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);
 
@@ -665,12 +655,10 @@
         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);
@@ -744,14 +732,12 @@
 
         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);
 
@@ -803,20 +789,14 @@
 
 
         //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),
@@ -854,4 +834,9 @@
 
     }
     */
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 d75afc4..a0228d6 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.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;
@@ -31,17 +30,17 @@
 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.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;
 
 
@@ -52,7 +51,7 @@
 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);
@@ -82,27 +81,29 @@
         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(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(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(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 +111,7 @@
         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 +127,38 @@
             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));
+        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(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,103 +171,80 @@
         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(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));
+        prism.getFeedHelper().delete(feed);
+        InstancesResult responseInstance = prism.getFeedHelper().getProcessInstanceStatus(feedName,
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 110));
 
         logger.info(responseInstance.getMessage());
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 7e2aa4e..bd786f4 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.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;
@@ -47,6 +46,7 @@
 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;
@@ -71,12 +71,11 @@
     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)
@@ -136,9 +135,7 @@
 
         //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);
@@ -215,9 +212,7 @@
 
         //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);
@@ -307,9 +302,7 @@
 
         //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);
@@ -360,4 +353,8 @@
         AssertUtil.checkForListSizes(cluster1ReplicatedData, cluster2ReplicatedData);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 738d0d1..f7a706d 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,17 @@
 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;
 
 /**
@@ -49,7 +49,6 @@
     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)
@@ -58,7 +57,6 @@
         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();
     }
@@ -75,12 +73,11 @@
      */
     @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);
@@ -94,7 +91,7 @@
      */
     @Test(groups = {"singleCluster"})
     public void resumeNonExistentFeed() throws Exception {
-        AssertUtil.assertFailed(feedHelper.resume(URLS.RESUME_URL, feed));
+        AssertUtil.assertFailed(feedHelper.resume(feed));
     }
 
     /**
@@ -104,10 +101,9 @@
      */
     @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));
     }
 
     /**
@@ -117,13 +113,17 @@
      */
     @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);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 7ae5da4..8f0ee4e 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,16 +24,16 @@
 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.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;
 
 /**
@@ -45,20 +45,14 @@
     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(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();
     }
@@ -76,15 +70,15 @@
      */
     @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);
     }
@@ -97,11 +91,11 @@
     @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);
     }
@@ -113,14 +107,13 @@
      */
     @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);
     }
 
@@ -131,14 +124,13 @@
      */
     @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));
     }
 
     /**
@@ -148,6 +140,11 @@
      */
     @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)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 75def82..4ee229f 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,18 +25,18 @@
 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;
 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;
 
 /**
@@ -48,12 +48,8 @@
     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 {
@@ -61,11 +57,10 @@
         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 =
-            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();
     }
@@ -83,12 +78,11 @@
      */
     @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);
 
@@ -105,15 +99,14 @@
      */
     @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();
@@ -129,11 +122,11 @@
      */
     @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();
@@ -148,14 +141,13 @@
      */
     @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(
@@ -170,10 +162,15 @@
      */
     @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"));
 
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 a658102..2c246de 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,16 +25,15 @@
 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;
 import org.apache.falcon.regression.testHelper.BaseTestClass;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 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;
@@ -52,13 +51,8 @@
 
     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);
-
-    @BeforeMethod(alwaysRun = true)
-    public void uploadWorkflow() throws Exception {
-        uploadDirToClusters(aggregateWorkflowDir, OSUtil.RESOURCES_OOZIE);
-    }
+    private String feed;
 
     @BeforeMethod(alwaysRun = true)
     public void setUp(Method method) throws Exception {
@@ -66,7 +60,7 @@
         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)
@@ -90,9 +84,8 @@
     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, bundles[0].getDataSets().get(0));
+            .submitEntity(bundles[0].getClusters().get(0))).getStatusCode(), 200);
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
     }
 
@@ -109,17 +102,15 @@
 
         //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));
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(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);
     }
 
     /**
@@ -130,8 +121,7 @@
      */
     @Test(groups = {"singleCluster"})
     public void snsFeedWithoutCluster() throws Exception {
-        ServiceResponse response = prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0));
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertFailed(response);
     }
 
@@ -145,14 +135,10 @@
     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(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));
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.RUNNING);
     }
@@ -167,19 +153,16 @@
     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, bundles[0].getDataSets().get(0)))
-                .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, bundles[0].getDataSets().get(0));
-
+        ServiceResponse response = prism.getFeedHelper().submitAndSchedule(feed);
         AssertUtil.assertSucceeded(response);
         AssertUtil.checkStatus(clusterOC, EntityType.FEED, bundles[0], Job.Status.SUSPENDED);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
-
-
-
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 a8795db..3bd71ab 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,14 +24,14 @@
 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.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;
 
 /**
@@ -42,24 +42,18 @@
 
     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(baseAppHDFSDir, this.getClass().getSimpleName());
         bundles[0].generateUniqueBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
 
         //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,7 +70,7 @@
      */
     @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);
     }
 
@@ -87,13 +81,13 @@
      */
     @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);
     }
 
@@ -104,13 +98,13 @@
      */
     @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);
     }
 
@@ -121,10 +115,15 @@
      */
     @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);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 d324b64..adab5a6 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,16 +24,16 @@
 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.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;
 
 /**
@@ -45,24 +45,18 @@
     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(baseAppHDFSDir, this.getClass().getSimpleName());
         bundles[0].generateUniqueBundle();
         bundles[0] = new Bundle(bundles[0], cluster);
-        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
 
         //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();
@@ -81,11 +75,10 @@
      */
     @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);
     }
@@ -98,14 +91,13 @@
      */
     @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);
@@ -118,14 +110,13 @@
      */
     @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);
     }
 
@@ -136,7 +127,7 @@
      */
     @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);
     }
 
@@ -147,10 +138,15 @@
      */
     @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);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 4e51a79..86645a7 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.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;
@@ -57,24 +58,20 @@
      */
 
     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;
-
     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);
     }
 
@@ -91,7 +88,12 @@
             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,
@@ -103,12 +105,14 @@
             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,
@@ -120,16 +124,18 @@
             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,
@@ -141,15 +147,12 @@
             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)
@@ -160,4 +163,9 @@
             HadoopUtil.deleteDirIfExists(Util.getPathPrefix(feedInputPath), fs);
         }
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 2e62360..6c873c3 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,45 +50,34 @@
 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 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;
-
     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,
@@ -111,100 +100,88 @@
             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 {
@@ -215,39 +192,31 @@
             ClusterType.SOURCE, null, null);
         processBundle.submitFeedsScheduleProcess(prism);
         InstancesSummaryResult r = prism.getProcessHelper()
-            .getInstanceSummary(Util.readEntityName(processBundle.getProcessData()),
-                "?start=" + startTime);
+            .getInstanceSummary(processName, "?start=" + startTime);
 
-        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(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 {
@@ -256,7 +225,6 @@
         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,
@@ -271,8 +239,7 @@
             .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"),
@@ -290,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);
+            .getInstanceSummary(Util.readEntityName(feed), "?start=" + startTime);
 
-        r = prism.getFeedHelper()
-            .getInstanceSummary(Util.readEntityName(feed),
-                "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime,
-                    -20));
-
+        r = prism.getFeedHelper().getInstanceSummary(Util.readEntityName(feed),
+            "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(endTime, -20));
     }
 
-
     @AfterMethod(alwaysRun = true)
     public void tearDown() throws IOException {
         removeBundles(processBundle);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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..27d4fdf 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.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;
@@ -50,12 +49,14 @@
 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;
@@ -75,10 +76,9 @@
     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 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;
 
@@ -117,14 +117,14 @@
     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);
@@ -136,7 +136,7 @@
 
             //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,
@@ -170,13 +170,13 @@
     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);
@@ -186,7 +186,7 @@
             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);
 
@@ -228,13 +228,13 @@
     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);
@@ -244,7 +244,7 @@
             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);
@@ -281,13 +281,13 @@
     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);
@@ -296,7 +296,7 @@
             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,
@@ -337,13 +337,13 @@
     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);
@@ -352,7 +352,7 @@
             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);
@@ -386,13 +386,13 @@
     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 {
@@ -400,7 +400,7 @@
             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);
@@ -435,15 +435,14 @@
     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);
@@ -452,7 +451,7 @@
             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);
@@ -490,14 +489,14 @@
     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);
@@ -506,7 +505,7 @@
             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);
@@ -543,13 +542,13 @@
         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);
@@ -558,7 +557,7 @@
             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);
@@ -581,14 +580,14 @@
         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);
@@ -597,7 +596,7 @@
             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,
@@ -635,14 +634,14 @@
         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);
@@ -651,7 +650,7 @@
             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()),
@@ -692,12 +691,12 @@
         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);
@@ -706,7 +705,7 @@
             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;
@@ -732,7 +731,7 @@
             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));
@@ -763,7 +762,7 @@
 
             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,
@@ -789,12 +788,12 @@
         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);
@@ -806,7 +805,7 @@
                 Util.getHadoopDataFromDir(clusterFS, bundles[0].getInputFeedFromBundle(),
                     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;
@@ -867,13 +866,13 @@
         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 {
@@ -881,7 +880,7 @@
             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);
@@ -890,7 +889,7 @@
                 (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()
@@ -1185,5 +1184,9 @@
 
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
 
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 86faf3b..cf91d83 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;
@@ -35,11 +36,13 @@
 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;
 
@@ -54,33 +57,27 @@
     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(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);
-
         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 +89,9 @@
         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);
     }
 
@@ -110,16 +110,12 @@
         //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,24 +123,24 @@
         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);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 e404567..9a9c686 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,18 +32,19 @@
 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.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;
 
@@ -54,8 +55,7 @@
 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);
@@ -181,12 +181,11 @@
         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
@@ -218,8 +217,7 @@
         //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 ");
 
@@ -279,5 +277,10 @@
         AssertUtil.assertSucceeded(responseInstance);
         Assert.assertTrue(responseInstance.getInstances() != null);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
 
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 03fdd94..68b308e 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.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -55,42 +56,39 @@
     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 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;
+    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(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(baseAppHDFSDir, this.getClass().getSimpleName());
         bundles[0] = new Bundle(bundles[0], cluster);
         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());
     }
 
     @AfterMethod(alwaysRun = true)
@@ -108,15 +106,11 @@
     @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(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceKill(processName,
+            "?start=2010-01-02T01:00Z");
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.KILLED);
     }
 
@@ -133,14 +127,11 @@
         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);
         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);
     }
 
@@ -156,14 +147,11 @@
         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);
         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());
     }
@@ -187,18 +175,14 @@
         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);
-        bundles[0].setOutputFeedLocationData(feedOutputPath);
         bundles[0].setProcessConcurrency(6);
         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());
     }
 
@@ -214,16 +198,12 @@
          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);
         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);
     }
@@ -238,20 +218,15 @@
     @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(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");
-        InstanceUtil.validateResponse(result, 5, 2, 0, 0, 3);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        InstanceUtil.validateResponse(r, 5, 2, 0, 0, 3);
     }
 
     /**
@@ -263,20 +238,14 @@
     @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(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");
-        InstanceUtil.validateResponse(result, 5, 4, 0, 0, 1);
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceStatus(processName,
+            "?start=2010-01-02T01:00Z&end=2010-01-02T01:20Z");
+        InstanceUtil.validateResponse(r, 5, 4, 0, 0, 1);
     }
 
     /**
@@ -288,17 +257,11 @@
     @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(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?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);
     }
 
@@ -311,28 +274,17 @@
     @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), Util.getProcessName(bundles[0]
-            .getProcessData()), 1, CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
-        InstancesResult r = prism.getProcessHelper()
-            .getProcessInstanceKill(Util.readEntityName(bundles[0].getProcessData()),
-                "?start=2010-01-02T01:00Z");
+        InstanceUtil.waitTillInstanceReachState(serverOC.get(0), processName, 1,
+            CoordinatorAction.Status.SUCCEEDED, EntityType.PROCESS);
+        InstancesResult r = prism.getProcessHelper().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");
-        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
-
 }
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 44aff75..900a376 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.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -53,16 +54,16 @@
 
     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 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);
     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 {
@@ -74,11 +75,9 @@
         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)
@@ -89,6 +88,9 @@
         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();
     }
 
     @AfterMethod(alwaysRun = true)
@@ -106,22 +108,18 @@
     @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);
+        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 + "&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 + "&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 6, 5, 1, 0);
     }
 
@@ -133,22 +131,18 @@
     @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);
-        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 + "&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 + "&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 3, 3, 0, 0);
     }
 
@@ -161,21 +155,17 @@
     @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);
-        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 + "&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 + "&end=2010-01-02T01:11Z");
         TimeUtil.sleepSeconds(TIMEOUT);
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 6, 6, 0, 0);
     }
@@ -188,20 +178,14 @@
     @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);
-        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);
+        String wfID = InstanceUtil.getWorkflows(cluster, processName, Status.KILLED).get(0);
+        prism.getProcessHelper().getProcessInstanceRerun(processName, start);
         Assert.assertTrue(InstanceUtil.isWorkflowRunning(clusterOC, wfID));
     }
 
@@ -214,20 +198,16 @@
     @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);
-        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);
         Assert.assertTrue(InstanceUtil.isWorkflowRunning(clusterOC, wfID));
     }
 
@@ -240,19 +220,14 @@
     @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(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 + "&end=2010-01-02T01:06Z");
+        prism.getProcessHelper().getProcessInstanceRerun(processName,
+            start + "&end=2010-01-02T01:06Z");
+        Assert.assertEquals(InstanceUtil.getInstanceStatus(cluster, processName, 0, 1),
             CoordinatorAction.Status.SUSPENDED);
     }
 
@@ -264,19 +239,16 @@
     @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);
-        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 + "&end=2010-01-02T01:11Z");
         InstanceUtil.areWorkflowsRunning(clusterOC, wfIDs, 3, 3, 0, 0);
     }
 
@@ -290,31 +262,22 @@
     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);
         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 + "&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");
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 57b7d65..cfe55eb 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.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;
@@ -40,6 +39,7 @@
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -52,14 +52,14 @@
     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;
     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 {
@@ -70,11 +70,9 @@
         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)
@@ -86,6 +84,11 @@
         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)
@@ -102,25 +105,17 @@
      */
     @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);
     }
 
     /**
@@ -131,28 +126,18 @@
      */
     @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);
     }
 
     /**
@@ -163,28 +148,17 @@
      */
     @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);
     }
 
     /**
@@ -195,22 +169,15 @@
      */
     @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);
     }
 
@@ -222,15 +189,9 @@
      */
     @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);
     }
 
@@ -242,13 +203,8 @@
      */
     @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);
     }
 
@@ -260,15 +216,10 @@
      */
     @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");
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
+        InstancesResult r = prism.getProcessHelper().getProcessInstanceResume(processName,
+            "?start=2010-01-02T01:05Z");
         InstanceUtil.validateSuccessWithStatusCode(r, ResponseKeys.PROCESS_NOT_FOUND);
     }
 
@@ -279,23 +230,14 @@
      */
     @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);
     }
 
     /**
@@ -307,28 +249,18 @@
      */
     @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);
     }
 
     /**
@@ -339,38 +271,21 @@
      */
     @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)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-
-        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 422c6b3..8934eb7 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.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;
@@ -43,6 +42,7 @@
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -56,33 +56,27 @@
     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;
+    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(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());
@@ -95,6 +89,7 @@
         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)
@@ -113,15 +108,12 @@
         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(process));
         TimeUtil.sleepSeconds(TIMEOUT);
-        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(URLS.RESUME_URL,
-            bundles[0].getProcessData()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(process));
         TimeUtil.sleepSeconds(TIMEOUT);
-        InstancesResult r = prism.getProcessHelper()
-            .getRunningInstance(URLS.INSTANCE_RUNNING,
-                Util.readEntityName(bundles[0].getProcessData()));
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -135,12 +127,9 @@
     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,
-                Util.readEntityName(bundles[0].getProcessData()));
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         InstanceUtil.validateSuccessWOInstances(r);
     }
 
@@ -153,9 +142,7 @@
     @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(processName);
         InstanceUtil.validateSuccess(r, bundles[0], WorkflowStatus.RUNNING);
     }
 
@@ -166,9 +153,7 @@
      */
     @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");
     }
@@ -181,10 +166,8 @@
     @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,
-                Util.readEntityName(bundles[0].getProcessData()));
+        prism.getProcessHelper().delete(bundles[0].getProcessData());
+        InstancesResult r = prism.getProcessHelper().getRunningInstance(processName);
         Assert.assertEquals(r.getStatusCode(), ResponseKeys.PROCESS_NOT_FOUND,
             "Unexpected status code");
     }
@@ -198,22 +181,13 @@
     @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(processName);
+        InstanceUtil.validateSuccessWOInstances(r);
     }
 
-
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle b = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-        b = new Bundle(b, cluster);
-        b.setInputFeedDataPath(feedInputPath);
-        String prefix = b.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 4f06501..df959b5 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,14 +32,15 @@
 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;
 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;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -61,40 +62,37 @@
     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;
+    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(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);
     }
 
-
+    /**
+     *  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());
@@ -103,6 +101,9 @@
         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)
@@ -124,9 +125,8 @@
         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);
     }
@@ -144,9 +144,8 @@
         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);
     }
@@ -159,13 +158,10 @@
      */
     @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);
     }
 
@@ -177,10 +173,8 @@
     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);
     }
 
@@ -192,11 +186,8 @@
      */
     @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);
     }
@@ -209,12 +200,9 @@
      */
     @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);
     }
@@ -227,13 +215,9 @@
      */
     @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()),
+        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)) {
             Assert.assertTrue(false);
@@ -248,14 +232,10 @@
      */
     @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()));
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(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);
     }
@@ -268,11 +248,8 @@
      */
     @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);
     }
@@ -285,18 +262,14 @@
      */
     @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);
     }
 
@@ -308,24 +281,19 @@
      */
     @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(process);
+        AssertUtil.checkStatus(clusterOC, EntityType.PROCESS, process, Job.Status.SUSPENDED);
+        prism.getProcessHelper().resume(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);
     }
 
@@ -338,14 +306,11 @@
      */
     @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);
     }
 
@@ -358,7 +323,6 @@
     @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");
@@ -375,23 +339,18 @@
      */
     @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(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);
     }
 
@@ -404,10 +363,8 @@
     @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);
     }
 
@@ -421,17 +378,20 @@
     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);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 4cbaf7d..9ba6ceb 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,12 +32,12 @@
 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;
 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;
@@ -45,6 +45,7 @@
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.List;
 
@@ -55,15 +56,15 @@
 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);
     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 {
@@ -75,11 +76,10 @@
         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)
@@ -90,6 +90,9 @@
         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)
@@ -106,23 +109,17 @@
     @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);
     }
@@ -136,15 +133,11 @@
     @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);
     }
@@ -158,23 +151,17 @@
     @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);
     }
@@ -188,15 +175,11 @@
     @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);
     }
 
@@ -209,23 +192,17 @@
     @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);
     }
@@ -238,15 +215,11 @@
     @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);
@@ -262,24 +235,16 @@
     @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(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(processName);
     }
 
     /**
@@ -292,34 +257,22 @@
     @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);
     }
 
     @AfterClass(alwaysRun = true)
-    public void deleteData() throws Exception {
-        LOGGER.info("in @AfterClass");
-        Bundle bundle = BundleUtil.readELBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-        bundle = new Bundle(bundle, cluster);
-        bundle.setInputFeedDataPath(feedInputPath);
-        String prefix = bundle.getFeedDataPathPrefix();
-        HadoopUtil.deleteDirIfExists(prefix.substring(1), clusterFS);
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 b64fca4..743990e 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.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;
 
@@ -52,45 +54,42 @@
     String testDir = baseHDFSDir + "/ProcessLibPath";
     String testLibDir = testDir + "/TestLib";
     private static final Logger logger = Logger.getLogger(ProcessLibPathTest.class);
+    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(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(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);
-        bundles[0].setOutputFeedLocationData(
-            baseHDFSDir + "/output-data/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}");
+        bundles[0].setOutputFeedLocationData(baseHDFSDir + "/output-data" + MINUTE_DATE_PATTERN);
         bundles[0].setProcessConcurrency(1);
         bundles[0].setProcessLibPath(testLibDir);
+        process = bundles[0].getProcessData();
+        processName = Util.readEntityName(process);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -109,13 +108,11 @@
         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 +128,15 @@
         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);
+    }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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..5598744
--- /dev/null
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
@@ -0,0 +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(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);
+    }
+}
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 3244dc9..b3a46a2 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.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;
@@ -648,4 +650,9 @@
     public void tearDown() {
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 c960ed6..ce94883 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.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;
@@ -181,10 +182,7 @@
             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
@@ -276,10 +274,7 @@
             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
@@ -356,4 +351,9 @@
     public void tearDown() {
         removeBundles();
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 557bd43..68924cc 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.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;
@@ -46,6 +45,7 @@
 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;
@@ -105,7 +105,7 @@
             .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 +127,11 @@
             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);
@@ -288,4 +287,8 @@
         return MathUtil.crossProduct(periods, retentionUnits, dataTypes);
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 1ef35ce..432e682 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.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;
@@ -58,9 +60,8 @@
     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;
@@ -225,4 +226,8 @@
 
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 53327eb..ed0282e 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,16 +38,17 @@
 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;
 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;
@@ -55,7 +56,6 @@
 
 @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 =
@@ -90,28 +90,25 @@
         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());
         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()));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed.toString()));
         }
 
         FeedMerlin outputMerlin = new FeedMerlin(bundles[0].getOutputFeedFromBundle());
         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()));
+            AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(feed.toString()));
         }
     }
 
@@ -606,7 +603,8 @@
 
     @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 +651,9 @@
             AssertUtil.checkForListSize(userIncoming.filterByName(feed.getName()), 1);
         }
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 f27aca1..d3bfc7d 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.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;
 
 /*
@@ -49,10 +51,8 @@
     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);
 
@@ -83,68 +83,71 @@
     }
 
     /**
-     *
      * tries to submit process with invalid el exp
      */
     @Test(groups = {"singleCluster"})
     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);
     }
 
     /**
-     *
      * tries to update process with invalid EL exp
      */
     @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);
+            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");
     }
 
     /**
      * tries to submit feed with invalied EL exp
-     *
      */
     @Test(groups = {"singleCluster"})
     public void testDryRunFailureScheduleFeed() throws Exception {
         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);
     }
 
     /**
-     *
      * 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(feed);
         AssertUtil.assertSucceeded(response);
         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 : 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();
     }
 }
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 f5bee03..7e3bcbb 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,15 +26,16 @@
 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;
+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")
@@ -44,7 +45,7 @@
     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);
 
@@ -115,7 +116,11 @@
         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)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
     }
 }
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 25a8789..e69de29 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
@@ -1,219 +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.AfterMethod;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-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(baseAppHDFSDir, this.getClass().getSimpleName());
-        bundles[0].generateUniqueBundle();
-        bundles[0] = new Bundle(bundles[0], cluster1);
-        bundles[0].setProcessWorkflow(impressionrcWorkflowDir);
-
-        bundles[1] =
-            BundleUtil.readImpressionRCBundle(baseAppHDFSDir, this.getClass().getSimpleName());
-        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}";
-
-        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, bundles[0].getClusters().get(0));
-        prism.getClusterHelper().submitEntity(URLS.SUBMIT_URL, bundles[1].getClusters().get(0));
-
-        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),
-            Util.readEntityName(bundles[0].getClusters().get(0)), 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),
-            Util.readEntityName(bundles[1].getClusters().get(0)), 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),
-            Util.readEntityName(bundles[0].getClusters().get(0)), 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),
-            Util.readEntityName(bundles[1].getClusters().get(0)), 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),
-            Util.readEntityName(bundles[0].getClusters().get(0)), 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);
-
-        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,
-            Util.readEntityName(bundles[0].getClusters().get(0)),
-            XmlUtil.createProcessValidity(TimeUtil.getTimeWrtSystemTime(-2),
-                TimeUtil.getTimeWrtSystemTime(5)));
-        process = InstanceUtil.setProcessCluster(process,
-            Util.readEntityName(bundles[1].getClusters().get(0)),
-            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));
-    }
-
-}
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 26fd43e..1ceab75 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.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;
@@ -52,6 +51,7 @@
 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;
@@ -72,7 +72,7 @@
 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";
@@ -129,8 +129,7 @@
         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);
@@ -182,8 +181,7 @@
         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);
 
@@ -256,8 +254,7 @@
         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);
@@ -341,8 +338,7 @@
         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);
@@ -381,8 +377,7 @@
         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,
@@ -416,8 +411,7 @@
         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,
@@ -499,8 +493,7 @@
         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);
 
@@ -586,14 +579,12 @@
         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);
@@ -601,8 +592,7 @@
                 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],
@@ -621,8 +611,7 @@
         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(),
@@ -681,8 +670,7 @@
         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
@@ -782,8 +770,7 @@
         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);
 
@@ -851,8 +838,7 @@
         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);
 
@@ -872,8 +858,7 @@
         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();
@@ -885,8 +870,7 @@
             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(),
@@ -928,8 +912,7 @@
         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);
 
@@ -949,8 +932,7 @@
         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()
@@ -982,8 +964,7 @@
         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);
 
@@ -1002,10 +983,8 @@
         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()
@@ -1018,8 +997,7 @@
                 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);
 
@@ -1049,8 +1027,7 @@
 
         //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);
 
@@ -1063,8 +1040,7 @@
                 EntityType.PROCESS);
 
         //submit new feed
-        AssertUtil.assertSucceeded(
-                prism.getFeedHelper().submitEntity(URLS.SUBMIT_URL, inputFeed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitEntity(inputFeed));
 
         Util.shutDownService(cluster3.getProcessHelper());
 
@@ -1113,8 +1089,7 @@
         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);
 
@@ -1180,8 +1155,7 @@
         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);
@@ -1199,8 +1173,7 @@
             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()))
@@ -1208,8 +1181,7 @@
             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());
 
@@ -1248,8 +1220,7 @@
         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
@@ -1300,8 +1271,7 @@
         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);
 
@@ -1345,8 +1315,7 @@
         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);
 
@@ -1384,9 +1353,7 @@
         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);
@@ -1409,9 +1376,7 @@
         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()
@@ -1446,8 +1411,7 @@
         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);
@@ -1466,13 +1430,11 @@
         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);
 
@@ -1578,8 +1540,7 @@
 
     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 {
@@ -1706,4 +1667,8 @@
         return null;
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 3965e2d..e1bc38e 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.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;
 
@@ -81,28 +83,24 @@
      */
     @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 +113,30 @@
      */
     @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 +151,24 @@
     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 +184,15 @@
         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 +202,8 @@
         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 +215,19 @@
      */
     @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 +244,43 @@
         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 +297,47 @@
         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);
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 24f0068..c77efb1 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.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;
@@ -74,7 +76,7 @@
     @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();
@@ -89,7 +91,7 @@
 
         //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();
@@ -126,7 +128,7 @@
         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();
@@ -145,7 +147,7 @@
 
         //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();
@@ -176,7 +178,7 @@
         //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();
@@ -203,9 +205,9 @@
     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();
@@ -219,7 +221,7 @@
         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();
@@ -254,12 +256,12 @@
         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();
@@ -277,7 +279,7 @@
 
         //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();
@@ -322,7 +324,7 @@
 
         //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();
@@ -373,7 +375,7 @@
 
         //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();
@@ -403,7 +405,7 @@
 
         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();
@@ -432,4 +434,8 @@
             store1.toArray(new String[store1.size()])), "DataStores are not equal!");
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 45997bf..d4ba566 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.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;
 
 
@@ -78,7 +80,7 @@
      */
     @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],
@@ -102,9 +104,9 @@
      */
     @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],
@@ -179,11 +181,11 @@
     @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] =
@@ -215,15 +217,15 @@
     @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++) {
@@ -267,4 +269,8 @@
         }
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 9974062..6efbdb0 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,16 +32,17 @@
 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;
 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;
@@ -105,8 +106,7 @@
         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();
@@ -158,8 +158,7 @@
         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();
@@ -190,8 +189,7 @@
 
         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();
@@ -217,8 +215,7 @@
     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();
@@ -231,8 +228,7 @@
         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();
@@ -272,8 +268,7 @@
 
 
         //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();
@@ -290,8 +285,7 @@
         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();
@@ -335,8 +329,7 @@
         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();
@@ -375,13 +368,12 @@
         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";
@@ -397,19 +389,17 @@
             .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());
 
-        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(Util.URLS.DELETE_URL, feed));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().delete(feed));
     }
 
 
@@ -430,8 +420,7 @@
         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 @@
         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 @@
         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 @@
         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 @@
         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 @@
         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 @@
         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 @@
 
         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 @@
         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 @@
         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 @@
 
         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 @@
         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 @@
 
 
         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 @@
         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";
@@ -834,23 +808,21 @@
             .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));
 
-        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);
 
@@ -867,14 +839,14 @@
 
         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)));
@@ -916,13 +888,12 @@
         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";
@@ -937,22 +908,20 @@
             .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));
 
-        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);
 
@@ -967,13 +936,13 @@
         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)));
@@ -1030,8 +999,7 @@
         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();
@@ -1062,8 +1030,7 @@
 
         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));
 
@@ -1125,4 +1092,8 @@
         return temp;
     }
 
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 b5c8cb2..c16d245 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,18 +29,19 @@
 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.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;
 
@@ -54,8 +55,7 @@
     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);
 
@@ -126,7 +126,7 @@
 
         logger.info("feed: " + Util.prettyPrintXml(feed));
 
-        prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
 
         String bundleId =
@@ -199,7 +199,7 @@
 
         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();
@@ -392,7 +392,7 @@
         //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
@@ -594,7 +594,7 @@
         //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
@@ -670,4 +670,9 @@
                 0) == 1,
             "id have to be equal 1");
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 a126a5d..0678595 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.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;
@@ -40,6 +39,7 @@
 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;
@@ -71,7 +71,6 @@
     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 @@
             .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"),
@@ -231,7 +230,7 @@
 
         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");
@@ -269,21 +268,21 @@
             .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));
 
-        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);
 
@@ -364,18 +363,17 @@
             .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));
 
-        ServiceResponse r =
-            prism.getFeedHelper().submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, feed);
+        ServiceResponse r = prism.getFeedHelper().submitAndSchedule(feed);
         TimeUtil.sleepSeconds(10);
         AssertUtil.assertSucceeded(r);
 
@@ -440,7 +438,7 @@
 
 
         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"),
@@ -467,11 +465,11 @@
 
         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,
@@ -547,14 +545,14 @@
             .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"),
@@ -564,7 +562,7 @@
         //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 " +
@@ -594,7 +592,7 @@
 
         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 +602,13 @@
             .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"),
@@ -620,11 +618,11 @@
 
         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,
@@ -705,27 +703,27 @@
                 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));
 
-        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);
 
@@ -777,7 +775,7 @@
         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,27 +785,27 @@
             .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));
 
-        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);
@@ -876,13 +874,13 @@
             .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"),
@@ -891,9 +889,14 @@
 
         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"));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 9b7f8fe..93d82e5 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,13 +30,13 @@
 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.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;
@@ -58,10 +58,9 @@
     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);
 
@@ -142,8 +141,8 @@
 
         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);
@@ -212,10 +211,8 @@
             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);
@@ -266,9 +263,9 @@
             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
 
@@ -293,8 +290,7 @@
             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 ");
 
@@ -308,4 +304,9 @@
         logger.info("updated feed: " + Util.prettyPrintXml(feed01));
         AssertUtil.assertSucceeded(prism.getFeedHelper().update(feed01, feed01));
     }
+
+    @AfterClass(alwaysRun = true)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 f137204..4902e3f 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.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")
@@ -81,38 +83,36 @@
         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);
     }
@@ -124,27 +124,21 @@
         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);
     }
 
@@ -154,25 +148,23 @@
         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);
@@ -182,7 +174,7 @@
         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);
         }
@@ -190,7 +182,7 @@
         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);
         }
@@ -198,15 +190,11 @@
 
     @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(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[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster1.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
+        AssertUtil.assertFailed(cluster2.getFeedHelper().resume(bundles[0].getDataSets().get(0)));
     }
 
     @Test
@@ -214,15 +202,11 @@
         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"})
@@ -233,28 +217,26 @@
         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);
     }
@@ -268,36 +250,29 @@
         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);
     }
@@ -308,12 +283,9 @@
 
         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"})
@@ -325,12 +297,9 @@
 
         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)));
     }
 
 
@@ -340,16 +309,15 @@
             .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)
+    public void tearDownClass() throws IOException {
+        cleanTestDirs();
+    }
 }
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 d7181ff..322d66c 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,11 +24,11 @@
 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;
 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;
@@ -81,8 +81,7 @@
         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);
@@ -90,4 +89,9 @@
         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();
+    }
 }
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 f14e1cc..574131d 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,18 +31,19 @@
 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;
 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 {
@@ -54,6 +55,7 @@
     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 {
@@ -70,6 +72,8 @@
             bundles[i].generateUniqueBundle();
             bundles[i].setProcessWorkflow(aggregateWorkflowDir);
         }
+        feed1 = bundles[0].getDataSets().get(0);
+        feed2 = bundles[1].getDataSets().get(0);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -80,22 +84,27 @@
         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
         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
@@ -110,102 +119,96 @@
         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)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         //ensure only one bundle is there
         Assert.assertEquals(OozieUtil.getBundles(cluster1OC,
-            Util.readEntityName(bundles[0].getDataSets().get(0)), EntityType.FEED).size(), 1);
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+            Util.readEntityName(feed1), EntityType.FEED).size(), 1);
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(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(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)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(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(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(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)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(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(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(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(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)));
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(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)));
-        AssertUtil.assertFailed(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertFailed(prism.getFeedHelper().submitAndSchedule(feed1));
+        AssertUtil.assertFailed(prism.getFeedHelper().submitAndSchedule(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(
+            bundles[1].getClusters().get(0)));
 
         Util.shutDownService(cluster1.getFeedHelper());
-
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+        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);
@@ -213,24 +216,27 @@
         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)));
+        AssertUtil.assertFailed(prism.getFeedHelper().submitAndSchedule(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)));
+        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);
@@ -239,11 +245,14 @@
         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)));
+        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);
@@ -252,108 +261,99 @@
         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))));
+        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, bundles[1].getDataSets().get(0)));
+        result = Util.parseResponse(cluster2.getFeedHelper().submitAndSchedule(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)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
         bundles[1].submitFeed();
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
 
-        AssertUtil.assertSucceeded(cluster1.getFeedHelper()
-            .suspend(URLS.SUSPEND_URL, bundles[0].getDataSets().get(0)));
+        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, bundles[0].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(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(feed1));
 
-        AssertUtil.assertSucceeded(cluster2.getFeedHelper().suspend(URLS.SUSPEND_URL,
-            bundles[1].getDataSets().get(0)));
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[0].getDataSets().get(0)));
+        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);
     }
 
-
+    /**
+     *  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(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(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)));
-        AssertUtil.assertSucceeded(prism.getFeedHelper()
-            .submitAndSchedule(URLS.SUBMIT_AND_SCHEDULE_URL, bundles[1].getDataSets().get(0)));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed1));
+        AssertUtil.assertSucceeded(prism.getFeedHelper().submitAndSchedule(feed2));
 
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper()
-                .getStatus(URLS.STATUS_URL, bundles[0].getDataSets().get(0))).getMessage(),
-            cluster1.getClusterHelper().getColoName() + "/RUNNING");
-
-        Assert.assertEquals(Util.parseResponse(prism.getFeedHelper()
-                .getStatus(URLS.STATUS_URL, bundles[1].getDataSets().get(0))).getMessage(),
-