You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by lx...@apache.org on 2016/08/17 04:26:57 UTC

[01/33] helix git commit: Fix task framework unit tests.

Repository: helix
Updated Branches:
  refs/heads/helix-0.6.x fe540ac9e -> 947a7d557


Fix task framework unit tests.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/760f8e3c
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/760f8e3c
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/760f8e3c

Branch: refs/heads/helix-0.6.x
Commit: 760f8e3caca39aeb53a171303ec24334cd59bf47
Parents: fe540ac
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Jan 19 18:15:07 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:33:42 2016 -0700

----------------------------------------------------------------------
 .../helix/integration/task/TaskTestUtil.java    | 16 +++++++
 .../task/TestRunJobsWithMissingTarget.java      | 20 +--------
 .../task/TestTaskRebalancerStopResume.java      | 46 ++++++++------------
 3 files changed, 36 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/760f8e3c/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index c5dd099..11677b8 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -247,6 +247,22 @@ public class TaskTestUtil {
     return buildRecurrentJobQueue(jobQueueName, 0);
   }
 
+  public static JobQueue.Builder buildJobQueue(String jobQueueName, int delayStart) {
+    Map<String, String> cfgMap = new HashMap<String, String>();
+    cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(120000));
+    Calendar cal = Calendar.getInstance();
+    cal.set(Calendar.MINUTE, cal.get(Calendar.MINUTE) + delayStart / 60);
+    cal.set(Calendar.SECOND, cal.get(Calendar.SECOND) + delayStart % 60);
+    cal.set(Calendar.MILLISECOND, 0);
+    cfgMap.put(WorkflowConfig.START_TIME,
+        WorkflowConfig.getDefaultDateFormat().format(cal.getTime()));
+    return new JobQueue.Builder(jobQueueName).fromMap(cfgMap);
+  }
+
+  public static JobQueue.Builder buildJobQueue(String jobQueueName) {
+    return buildJobQueue(jobQueueName, 0);
+  }
+
   public static boolean pollForParticipantParallelState() {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/760f8e3c/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index 74a8610..d8d2b60 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -146,28 +146,12 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     _manager.disconnect();
   }
 
-  private JobQueue.Builder buildJobQueue(String jobQueueName, int delayStart) {
-    Map<String, String> cfgMap = new HashMap<String, String>();
-    cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(120000));
-    Calendar cal = Calendar.getInstance();
-    cal.set(Calendar.MINUTE, cal.get(Calendar.MINUTE) + delayStart / 60);
-    cal.set(Calendar.SECOND, cal.get(Calendar.SECOND) + delayStart % 60);
-    cal.set(Calendar.MILLISECOND, 0);
-    cfgMap.put(WorkflowConfig.START_TIME,
-        WorkflowConfig.getDefaultDateFormat().format(cal.getTime()));
-    return new JobQueue.Builder(jobQueueName).fromMap(cfgMap);
-  }
-
-  private JobQueue.Builder buildJobQueue(String jobQueueName) {
-    return buildJobQueue(jobQueueName, 0);
-  }
-
   @Test public void testJobFailsWithMissingTarget() throws Exception {
     String queueName = TestHelper.getTestMethodName();
 
     // Create a queue
     LOG.info("Starting job-queue: " + queueName);
-    JobQueue.Builder queueBuilder = buildJobQueue(queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
     for (int i = 0; i < num_dbs; i++) {
@@ -192,7 +176,7 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
 
     // Create a queue
     LOG.info("Starting job-queue: " + queueName);
-    JobQueue.Builder queueBuilder = buildJobQueue(queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
     for (int i = 0; i < num_dbs; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/760f8e3c/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
index b67fa90..471d130 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
@@ -20,7 +20,6 @@ package org.apache.helix.integration.task;
  */
 
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
@@ -269,24 +268,25 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     // Create a queue
     LOG.info("Starting job-queue: " + queueName);
-    JobQueue queue = new JobQueue.Builder(queueName).build();
-    _driver.createQueue(queue);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
 
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
     for (int i = 0; i <= 4; i++) {
       String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
 
-      JobConfig.Builder job =
+      JobConfig.Builder jobBuilder =
           new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
               .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
               .setTargetPartitionStates(Sets.newHashSet(targetPartition));
       String jobName = targetPartition.toLowerCase() + "Job" + i;
       LOG.info("Enqueuing job: " + jobName);
-      _driver.enqueueJob(queueName, jobName, job);
+      queueBuilder.enqueueJob(jobName, jobBuilder);
       currentJobNames.add(i, jobName);
     }
 
+    _driver.createQueue(queueBuilder.build());
+
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", queueName, deletedJob1);
@@ -365,25 +365,13 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     }
   }
 
-  private JobQueue buildRecurrentJobQueue(String jobQueueName)
-  {
-    Map<String, String> cfgMap = new HashMap<String, String>();
-    cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(50000));
-    cfgMap.put(WorkflowConfig.START_TIME,
-        WorkflowConfig.getDefaultDateFormat().format(Calendar.getInstance().getTime()));
-    cfgMap.put(WorkflowConfig.RECURRENCE_INTERVAL, String.valueOf(60));
-    cfgMap.put(WorkflowConfig.RECURRENCE_UNIT, "SECONDS");
-    return (new JobQueue.Builder(jobQueueName).fromMap(cfgMap)).build();
-  }
-
   @Test
   public void stopDeleteJobAndResumeRecurrentQueue() throws Exception {
     String queueName = TestHelper.getTestMethodName();
 
     // Create a queue
     LOG.info("Starting job-queue: " + queueName);
-    JobQueue queue = buildRecurrentJobQueue(queueName);
-    _driver.createQueue(queue);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildRecurrentJobQueue(queueName);
 
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
@@ -398,10 +386,12 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
               .setTargetPartitionStates(Sets.newHashSet(targetPartition));
       String jobName = targetPartition.toLowerCase() + "Job" + i;
       LOG.info("Enqueuing job: " + jobName);
-      _driver.enqueueJob(queueName, jobName, job);
+      queueBuilder.enqueueJob(jobName, job);
       currentJobNames.add(i, jobName);
     }
 
+    _driver.createQueue(queueBuilder.build());
+
     WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
 
@@ -471,31 +461,31 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     // Create a queue
     LOG.info("Starting job-queue: " + queueName);
-    JobQueue queue = buildRecurrentJobQueue(queueName);
-    _driver.createQueue(queue);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildRecurrentJobQueue(queueName);
 
-    // create jobs
     List<JobConfig.Builder> jobs = new ArrayList<JobConfig.Builder>();
     List<String> jobNames = new ArrayList<String>();
     Map<String, String> commandConfig = ImmutableMap.of(TIMEOUT_CONFIG, String.valueOf(500));
 
-    final int JOB_COUNTS = 3;
 
+    int JOB_COUNTS = 3;
     for (int i = 0; i < JOB_COUNTS; i++) {
       String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
+      String jobName = targetPartition.toLowerCase() + "Job" + i;
 
       JobConfig.Builder job = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
           .setJobCommandConfigMap(commandConfig).setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
           .setTargetPartitionStates(Sets.newHashSet(targetPartition));
       jobs.add(job);
-      jobNames.add(targetPartition.toLowerCase() + "Job" + i);
+      jobNames.add(jobName);
     }
 
-    // enqueue all jobs except last one
-    for (int i = 0; i < JOB_COUNTS - 1; ++i) {
-      LOG.info("Enqueuing job: " + jobNames.get(i));
-      _driver.enqueueJob(queueName, jobNames.get(i), jobs.get(i));
+    for (int i = 0; i < JOB_COUNTS -1; i++) {
+      queueBuilder.enqueueJob(jobNames.get(i), jobs.get(i));
     }
+
+    _driver.createQueue(queueBuilder.build());
+
     String currentLastJob = jobNames.get(JOB_COUNTS - 2);
 
     WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);


[07/33] helix git commit: Refactor TaskUtil class to move as many as methods out of the class, and make other methods in it as internal API as possible. Expose necessary APIs in TaskDriver instead.

Posted by lx...@apache.org.
Refactor TaskUtil class to move as many as methods out of the class, and make other methods in it as internal API as possible. Expose necessary APIs in TaskDriver instead.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/579d82fd
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/579d82fd
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/579d82fd

Branch: refs/heads/helix-0.6.x
Commit: 579d82fd2aa8fdce8ec0e0c4d6da73cb8209729d
Parents: 6d42db4
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Feb 23 17:32:35 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:44:32 2016 -0700

----------------------------------------------------------------------
 .../org/apache/helix/task/JobRebalancer.java    |   6 +-
 .../org/apache/helix/task/TaskRebalancer.java   |  36 ++++
 .../java/org/apache/helix/task/TaskRunner.java  |  37 +++-
 .../java/org/apache/helix/task/TaskUtil.java    | 178 +++----------------
 .../apache/helix/task/WorkflowRebalancer.java   |  89 +++++++++-
 .../helix/integration/task/TaskTestUtil.java    |  35 ++--
 .../task/TestDisableJobExternalView.java        |   2 +-
 .../task/TestIndependentTaskRebalancer.java     |  25 ++-
 .../integration/task/TestRecurringJobQueue.java |  36 ++--
 .../task/TestRunJobsWithMissingTarget.java      |  11 +-
 .../integration/task/TestTaskRebalancer.java    |  32 ++--
 .../task/TestTaskRebalancerFailover.java        |  12 +-
 .../task/TestTaskRebalancerParallel.java        |   2 +-
 .../task/TestTaskRebalancerRetryLimit.java      |   4 +-
 .../task/TestTaskRebalancerStopResume.java      |  78 ++++----
 .../integration/task/TestUpdateWorkflow.java    |  19 +-
 16 files changed, 306 insertions(+), 296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index 93d4689..7eeafc7 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -90,7 +90,7 @@ public class JobRebalancer extends TaskRebalancer {
     // The job is already in a final state (completed/failed).
     if (jobState == TaskState.FAILED || jobState == TaskState.COMPLETED) {
       LOG.info("Job " + jobName + " is failed or already completed, clean up IS.");
-      TaskUtil.cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobName);
+      cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobName);
       _scheduledRebalancer.removeScheduledRebalance(jobName);
       return buildEmptyAssignment(jobName, currStateOutput);
     }
@@ -340,7 +340,7 @@ public class JobRebalancer extends TaskRebalancer {
               addAllPartitions(allPartitions, partitionsToDropFromIs);
 
               // remove IdealState of this job
-              TaskUtil.cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
+              cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
               return buildEmptyAssignment(jobResource, currStateOutput);
             } else {
               skippedPartitions.add(pId);
@@ -376,7 +376,7 @@ public class JobRebalancer extends TaskRebalancer {
     if (isJobComplete(jobCtx, allPartitions, skippedPartitions, jobCfg)) {
       markJobComplete(jobResource, jobCtx, workflowConfig, workflowCtx);
       // remove IdealState of this job
-      TaskUtil.cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
+      cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
     }
 
     // Make additional task assignments if needed.

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
index f35ce69..b006efc 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
@@ -29,8 +29,10 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.stages.ClusterDataCache;
@@ -171,6 +173,40 @@ public abstract class TaskRebalancer implements Rebalancer, MappingCalculator {
     return (startTime == null || startTime.getTime() <= System.currentTimeMillis());
   }
 
+  /**
+   * Cleans up IdealState and external view associated with a job/workflow resource.
+   */
+  protected static void cleanupIdealStateExtView(HelixDataAccessor accessor, final String resourceName) {
+    LOG.info("Cleaning up idealstate and externalView for job: " + resourceName);
+
+    // Delete the ideal state itself.
+    PropertyKey isKey = accessor.keyBuilder().idealStates(resourceName);
+    if (accessor.getProperty(isKey) != null) {
+      if (!accessor.removeProperty(isKey)) {
+        LOG.error(String.format(
+            "Error occurred while trying to clean up resource %s. Failed to remove node %s from Helix.",
+            resourceName, isKey));
+      }
+    } else {
+      LOG.warn(String.format("Idealstate for resource %s does not exist.", resourceName));
+    }
+
+    // Delete dead external view
+    // because job is already completed, there is no more current state change
+    // thus dead external views removal will not be triggered
+    PropertyKey evKey = accessor.keyBuilder().externalView(resourceName);
+    if (accessor.getProperty(evKey) != null) {
+      if (!accessor.removeProperty(evKey)) {
+        LOG.error(String.format(
+            "Error occurred while trying to clean up resource %s. Failed to remove node %s from Helix.",
+            resourceName, evKey));
+      }
+    }
+
+    LOG.info(String
+        .format("Successfully clean up idealstate/externalView for resource %s.", resourceName));
+  }
+
   @Override public IdealState computeNewIdealState(String resourceName,
       IdealState currentIdealState, CurrentStateOutput currentStateOutput,
       ClusterDataCache clusterData) {

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java b/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
index 7b17043..1bf88ec 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
@@ -19,7 +19,10 @@ package org.apache.helix.task;
  * under the License.
  */
 
+import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
 import org.apache.helix.task.TaskResult.Status;
 import org.apache.log4j.Logger;
 
@@ -168,8 +171,8 @@ public class TaskRunner implements Runnable {
    */
   private void requestStateTransition(TaskPartitionState state) {
     boolean success =
-        TaskUtil.setRequestedState(_manager.getHelixDataAccessor(), _instance, _sessionId,
-            _taskName, _taskPartition, state);
+        setRequestedState(_manager.getHelixDataAccessor(), _instance, _sessionId, _taskName,
+            _taskPartition, state);
     if (!success) {
       LOG.error(String
           .format(
@@ -177,4 +180,34 @@ public class TaskRunner implements Runnable {
               state, _instance, _sessionId, _taskPartition));
     }
   }
+
+  /**
+   * Request a state change for a specific task.
+   *
+   * @param accessor  connected Helix data accessor
+   * @param instance  the instance serving the task
+   * @param sessionId the current session of the instance
+   * @param resource  the job name
+   * @param partition the task partition name
+   * @param state     the requested state
+   * @return true if the request was persisted, false otherwise
+   */
+  private static boolean setRequestedState(HelixDataAccessor accessor, String instance,
+      String sessionId, String resource, String partition, TaskPartitionState state) {
+    LOG.debug(
+        String.format("Requesting a state transition to %s for partition %s.", state, partition));
+    try {
+      PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+      PropertyKey key = keyBuilder.currentState(instance, sessionId, resource);
+      CurrentState currStateDelta = new CurrentState(resource);
+      currStateDelta.setRequestedState(partition, state.name());
+
+      return accessor.updateProperty(key, currStateDelta);
+    } catch (Exception e) {
+      LOG.error(String
+          .format("Error when requesting a state transition to %s for partition %s.", state,
+              partition), e);
+      return false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
index ca274d0..49622f3 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
@@ -55,15 +55,17 @@ import com.google.common.collect.Maps;
 public class TaskUtil {
   private static final Logger LOG = Logger.getLogger(TaskUtil.class);
   public static final String CONTEXT_NODE = "Context";
+
   /**
    * Parses job resource configurations in Helix into a {@link JobConfig} object.
+   * This method is internal API, please use TaskDriver.getJobConfig();
    *
    * @param accessor    Accessor to access Helix configs
    * @param jobResource The name of the job resource
    * @return A {@link JobConfig} object if Helix contains valid configurations for the job, null
    * otherwise.
    */
-  public static JobConfig getJobCfg(HelixDataAccessor accessor, String jobResource) {
+  protected static JobConfig getJobCfg(HelixDataAccessor accessor, String jobResource) {
     HelixProperty jobResourceConfig = getResourceConfig(accessor, jobResource);
     if (jobResourceConfig == null) {
       return null;
@@ -83,25 +85,27 @@ public class TaskUtil {
 
   /**
    * Parses job resource configurations in Helix into a {@link JobConfig} object.
+   * This method is internal API, please use TaskDriver.getJobConfig();
    *
    * @param manager     HelixManager object used to connect to Helix.
    * @param jobResource The name of the job resource.
    * @return A {@link JobConfig} object if Helix contains valid configurations for the job, null
    * otherwise.
    */
-  public static JobConfig getJobCfg(HelixManager manager, String jobResource) {
+  protected static JobConfig getJobCfg(HelixManager manager, String jobResource) {
     return getJobCfg(manager.getHelixDataAccessor(), jobResource);
   }
 
   /**
    * Parses workflow resource configurations in Helix into a {@link WorkflowConfig} object.
+   * This method is internal API, please use TaskDriver.getWorkflowConfig();
    *
    * @param accessor  Accessor to access Helix configs
    * @param workflow The name of the workflow.
    * @return A {@link WorkflowConfig} object if Helix contains valid configurations for the
    * workflow, null otherwise.
    */
-  public static WorkflowConfig getWorkflowCfg(HelixDataAccessor accessor, String workflow) {
+  protected static WorkflowConfig getWorkflowCfg(HelixDataAccessor accessor, String workflow) {
     HelixProperty workflowCfg = getResourceConfig(accessor, workflow);
     if (workflowCfg == null) {
       return null;
@@ -115,60 +119,32 @@ public class TaskUtil {
 
   /**
    * Parses workflow resource configurations in Helix into a {@link WorkflowConfig} object.
+   * This method is internal API, please use TaskDriver.getWorkflowConfig();
    *
    * @param manager          Helix manager object used to connect to Helix.
    * @param workflow The name of the workflow resource.
    * @return A {@link WorkflowConfig} object if Helix contains valid configurations for the
    * workflow, null otherwise.
    */
-  public static WorkflowConfig getWorkflowCfg(HelixManager manager, String workflow) {
+  protected static WorkflowConfig getWorkflowCfg(HelixManager manager, String workflow) {
     return getWorkflowCfg(manager.getHelixDataAccessor(), workflow);
   }
 
   /**
-   * Request a state change for a specific task.
-   *
-   * @param accessor  connected Helix data accessor
-   * @param instance  the instance serving the task
-   * @param sessionId the current session of the instance
-   * @param resource  the job name
-   * @param partition the task partition name
-   * @param state     the requested state
-   * @return true if the request was persisted, false otherwise
-   */
-  public static boolean setRequestedState(HelixDataAccessor accessor, String instance,
-      String sessionId, String resource, String partition, TaskPartitionState state) {
-    LOG.debug(
-        String.format("Requesting a state transition to %s for partition %s.", state, partition));
-    try {
-      PropertyKey.Builder keyBuilder = accessor.keyBuilder();
-      PropertyKey key = keyBuilder.currentState(instance, sessionId, resource);
-      CurrentState currStateDelta = new CurrentState(resource);
-      currStateDelta.setRequestedState(partition, state.name());
-
-      return accessor.updateProperty(key, currStateDelta);
-    } catch (Exception e) {
-      LOG.error(String
-          .format("Error when requesting a state transition to %s for partition %s.", state,
-              partition), e);
-      return false;
-    }
-  }
-
-  /**
    * Get a Helix configuration scope at a resource (i.e. job and workflow) level
    *
    * @param clusterName the cluster containing the resource
    * @param resource    the resource name
    * @return instantiated {@link HelixConfigScope}
    */
-  public static HelixConfigScope getResourceConfigScope(String clusterName, String resource) {
+  protected static HelixConfigScope getResourceConfigScope(String clusterName, String resource) {
     return new HelixConfigScopeBuilder(HelixConfigScope.ConfigScopeProperty.RESOURCE)
         .forCluster(clusterName).forResource(resource).build();
   }
 
   /**
-   * Get the runtime context of a single job
+   * Get the runtime context of a single job.
+   * This method is internal API, please use TaskDriver.getJobContext();
    *
    * @param propertyStore Property store for the cluster
    * @param jobResource   The name of the job
@@ -183,31 +159,34 @@ public class TaskUtil {
   }
 
   /**
-   * Get the runtime context of a single job
+   * Get the runtime context of a single job.
+   * This method is internal API, please use TaskDriver.getJobContext();
    *
    * @param manager     a connection to Helix
    * @param jobResource the name of the job
    * @return the {@link JobContext}, or null if none is available
    */
-  public static JobContext getJobContext(HelixManager manager, String jobResource) {
+  protected static JobContext getJobContext(HelixManager manager, String jobResource) {
     return getJobContext(manager.getHelixPropertyStore(), jobResource);
   }
 
   /**
    * Set the runtime context of a single job
+   * This method is internal API;
    *
    * @param manager     a connection to Helix
    * @param jobResource the name of the job
    * @param ctx         the up-to-date {@link JobContext} for the job
    */
-  public static void setJobContext(HelixManager manager, String jobResource, JobContext ctx) {
+  protected static void setJobContext(HelixManager manager, String jobResource, JobContext ctx) {
     manager.getHelixPropertyStore()
         .set(Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, jobResource, CONTEXT_NODE),
             ctx.getRecord(), AccessOption.PERSISTENT);
   }
 
   /**
-   * Get the runtime context of a single workflow
+   * Get the runtime context of a single workflow.
+   * This method is internal API, please use TaskDriver.getWorkflowContext();
    *
    * @param propertyStore    Property store of the cluster
    * @param workflowResource The name of the workflow
@@ -222,13 +201,14 @@ public class TaskUtil {
   }
 
   /**
-   * Get the runtime context of a single workflow
+   * Get the runtime context of a single workflow.
+   * This method is internal API, please use TaskDriver.getWorkflowContext();
    *
    * @param manager          a connection to Helix
    * @param workflowResource the name of the workflow
    * @return the {@link WorkflowContext}, or null if none is available
    */
-  public static WorkflowContext getWorkflowContext(HelixManager manager, String workflowResource) {
+  protected static WorkflowContext getWorkflowContext(HelixManager manager, String workflowResource) {
     return getWorkflowContext(manager.getHelixPropertyStore(), workflowResource);
   }
 
@@ -367,126 +347,12 @@ public class TaskUtil {
     return null;
   }
 
-  /**
-   * Create a new workflow based on an existing one
-   *
-   * @param manager          connection to Helix
-   * @param origWorkflowName the name of the existing workflow
-   * @param newWorkflowName  the name of the new workflow
-   * @param newStartTime     a provided start time that deviates from the desired start time
-   * @return the cloned workflow, or null if there was a problem cloning the existing one
-   */
-  public static Workflow cloneWorkflow(HelixManager manager, String origWorkflowName,
-      String newWorkflowName, Date newStartTime) {
-    // Read all resources, including the workflow and jobs of interest
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
-    Map<String, HelixProperty> resourceConfigMap =
-        accessor.getChildValuesMap(keyBuilder.resourceConfigs());
-    if (!resourceConfigMap.containsKey(origWorkflowName)) {
-      LOG.error("No such workflow named " + origWorkflowName);
-      return null;
-    }
-    if (resourceConfigMap.containsKey(newWorkflowName)) {
-      LOG.error("Workflow with name " + newWorkflowName + " already exists!");
-      return null;
-    }
-
-    // Create a new workflow with a new name
-    HelixProperty workflowConfig = resourceConfigMap.get(origWorkflowName);
-    Map<String, String> wfSimpleFields = workflowConfig.getRecord().getSimpleFields();
-    JobDag jobDag = JobDag.fromJson(wfSimpleFields.get(WorkflowConfig.DAG));
-    Map<String, Set<String>> parentsToChildren = jobDag.getParentsToChildren();
-    Workflow.Builder workflowBuilder = new Workflow.Builder(newWorkflowName);
-
-    // Set the workflow expiry
-    workflowBuilder.setExpiry(Long.parseLong(wfSimpleFields.get(WorkflowConfig.EXPIRY)));
-
-    // Set the schedule, if applicable
-    ScheduleConfig scheduleConfig;
-    if (newStartTime != null) {
-      scheduleConfig = ScheduleConfig.oneTimeDelayedStart(newStartTime);
-    } else {
-      scheduleConfig = parseScheduleFromConfigMap(wfSimpleFields);
-    }
-    if (scheduleConfig != null) {
-      workflowBuilder.setScheduleConfig(scheduleConfig);
-    }
-
-    // Add each job back as long as the original exists
-    Set<String> namespacedJobs = jobDag.getAllNodes();
-    for (String namespacedJob : namespacedJobs) {
-      if (resourceConfigMap.containsKey(namespacedJob)) {
-        // Copy over job-level and task-level configs
-        String job = getDenamespacedJobName(origWorkflowName, namespacedJob);
-        HelixProperty jobConfig = resourceConfigMap.get(namespacedJob);
-        Map<String, String> jobSimpleFields = jobConfig.getRecord().getSimpleFields();
-
-        JobConfig.Builder jobCfgBuilder = JobConfig.Builder.fromMap(jobSimpleFields);
-
-        jobCfgBuilder.setWorkflow(newWorkflowName); // overwrite workflow name
-        Map<String, Map<String, String>> rawTaskConfigMap = jobConfig.getRecord().getMapFields();
-        List<TaskConfig> taskConfigs = Lists.newLinkedList();
-        for (Map<String, String> rawTaskConfig : rawTaskConfigMap.values()) {
-          TaskConfig taskConfig = TaskConfig.from(rawTaskConfig);
-          taskConfigs.add(taskConfig);
-        }
-        jobCfgBuilder.addTaskConfigs(taskConfigs);
-        workflowBuilder.addJobConfig(job, jobCfgBuilder);
-
-        // Add dag dependencies
-        Set<String> children = parentsToChildren.get(namespacedJob);
-        if (children != null) {
-          for (String namespacedChild : children) {
-            String child = getDenamespacedJobName(origWorkflowName, namespacedChild);
-            workflowBuilder.addParentChildDependency(job, child);
-          }
-        }
-      }
-    }
-    return workflowBuilder.build();
-  }
-
   private static HelixProperty getResourceConfig(HelixDataAccessor accessor, String resource) {
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
     return accessor.getProperty(keyBuilder.resourceConfig(resource));
   }
 
   /**
-   * Cleans up IdealState and external view associated with a job/workflow resource.
-   */
-  public static void cleanupIdealStateExtView(HelixDataAccessor accessor, final String resourceName) {
-    LOG.info("Cleaning up idealstate and externalView for job: " + resourceName);
-
-    // Delete the ideal state itself.
-    PropertyKey isKey = accessor.keyBuilder().idealStates(resourceName);
-    if (accessor.getProperty(isKey) != null) {
-      if (!accessor.removeProperty(isKey)) {
-        LOG.error(String.format(
-            "Error occurred while trying to clean up resource %s. Failed to remove node %s from Helix.",
-            resourceName, isKey));
-      }
-    } else {
-      LOG.warn(String.format("Idealstate for resource %s does not exist.", resourceName));
-    }
-
-    // Delete dead external view
-    // because job is already completed, there is no more current state change
-    // thus dead external views removal will not be triggered
-    PropertyKey evKey = accessor.keyBuilder().externalView(resourceName);
-    if (accessor.getProperty(evKey) != null) {
-      if (!accessor.removeProperty(evKey)) {
-        LOG.error(String.format(
-            "Error occurred while trying to clean up resource %s. Failed to remove node %s from Helix.",
-            resourceName, evKey));
-      }
-    }
-
-    LOG.info(String
-        .format("Successfully clean up idealstate/externalView for resource %s.", resourceName));
-  }
-
-  /**
    * Extracts the partition id from the given partition name.
    *
    * @param pName

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
index db5426c..05b6dc6 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
@@ -19,6 +19,7 @@ package org.apache.helix.task;
  * under the License.
  */
 
+import com.google.common.collect.Lists;
 import org.I0Itec.zkclient.DataUpdater;
 import org.apache.helix.*;
 import org.apache.helix.controller.stages.ClusterDataCache;
@@ -264,8 +265,8 @@ public class WorkflowRebalancer extends TaskRebalancer {
         String newWorkflowName = workflow + "_" + df.format(new Date(timeToSchedule));
         LOG.debug("Ready to start workflow " + newWorkflowName);
         if (!newWorkflowName.equals(lastScheduled)) {
-          Workflow clonedWf = TaskUtil
-              .cloneWorkflow(_manager, workflow, newWorkflowName, new Date(timeToSchedule));
+          Workflow clonedWf =
+              cloneWorkflow(_manager, workflow, newWorkflowName, new Date(timeToSchedule));
           TaskDriver driver = new TaskDriver(_manager);
           try {
             // Start the cloned workflow
@@ -298,6 +299,86 @@ public class WorkflowRebalancer extends TaskRebalancer {
   }
 
   /**
+   * Create a new workflow based on an existing one
+   *
+   * @param manager          connection to Helix
+   * @param origWorkflowName the name of the existing workflow
+   * @param newWorkflowName  the name of the new workflow
+   * @param newStartTime     a provided start time that deviates from the desired start time
+   * @return the cloned workflow, or null if there was a problem cloning the existing one
+   */
+  public static Workflow cloneWorkflow(HelixManager manager, String origWorkflowName,
+      String newWorkflowName, Date newStartTime) {
+    // Read all resources, including the workflow and jobs of interest
+    HelixDataAccessor accessor = manager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    Map<String, HelixProperty> resourceConfigMap =
+        accessor.getChildValuesMap(keyBuilder.resourceConfigs());
+    if (!resourceConfigMap.containsKey(origWorkflowName)) {
+      LOG.error("No such workflow named " + origWorkflowName);
+      return null;
+    }
+    if (resourceConfigMap.containsKey(newWorkflowName)) {
+      LOG.error("Workflow with name " + newWorkflowName + " already exists!");
+      return null;
+    }
+
+    // Create a new workflow with a new name
+    HelixProperty workflowConfig = resourceConfigMap.get(origWorkflowName);
+    Map<String, String> wfSimpleFields = workflowConfig.getRecord().getSimpleFields();
+    JobDag jobDag = JobDag.fromJson(wfSimpleFields.get(WorkflowConfig.DAG));
+    Map<String, Set<String>> parentsToChildren = jobDag.getParentsToChildren();
+    Workflow.Builder workflowBuilder = new Workflow.Builder(newWorkflowName);
+
+    // Set the workflow expiry
+    workflowBuilder.setExpiry(Long.parseLong(wfSimpleFields.get(WorkflowConfig.EXPIRY)));
+
+    // Set the schedule, if applicable
+    ScheduleConfig scheduleConfig;
+    if (newStartTime != null) {
+      scheduleConfig = ScheduleConfig.oneTimeDelayedStart(newStartTime);
+    } else {
+      scheduleConfig = TaskUtil.parseScheduleFromConfigMap(wfSimpleFields);
+    }
+    if (scheduleConfig != null) {
+      workflowBuilder.setScheduleConfig(scheduleConfig);
+    }
+
+    // Add each job back as long as the original exists
+    Set<String> namespacedJobs = jobDag.getAllNodes();
+    for (String namespacedJob : namespacedJobs) {
+      if (resourceConfigMap.containsKey(namespacedJob)) {
+        // Copy over job-level and task-level configs
+        String job = TaskUtil.getDenamespacedJobName(origWorkflowName, namespacedJob);
+        HelixProperty jobConfig = resourceConfigMap.get(namespacedJob);
+        Map<String, String> jobSimpleFields = jobConfig.getRecord().getSimpleFields();
+
+        JobConfig.Builder jobCfgBuilder = JobConfig.Builder.fromMap(jobSimpleFields);
+
+        jobCfgBuilder.setWorkflow(newWorkflowName); // overwrite workflow name
+        Map<String, Map<String, String>> rawTaskConfigMap = jobConfig.getRecord().getMapFields();
+        List<TaskConfig> taskConfigs = Lists.newLinkedList();
+        for (Map<String, String> rawTaskConfig : rawTaskConfigMap.values()) {
+          TaskConfig taskConfig = TaskConfig.from(rawTaskConfig);
+          taskConfigs.add(taskConfig);
+        }
+        jobCfgBuilder.addTaskConfigs(taskConfigs);
+        workflowBuilder.addJobConfig(job, jobCfgBuilder);
+
+        // Add dag dependencies
+        Set<String> children = parentsToChildren.get(namespacedJob);
+        if (children != null) {
+          for (String namespacedChild : children) {
+            String child = TaskUtil.getDenamespacedJobName(origWorkflowName, namespacedChild);
+            workflowBuilder.addParentChildDependency(job, child);
+          }
+        }
+      }
+    }
+    return workflowBuilder.build();
+  }
+
+  /**
    * Cleans up workflow configs and workflow contexts associated with this workflow,
    * including all job-level configs and context, plus workflow-level information.
    */
@@ -319,7 +400,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
     // clean up workflow-level info if this was the last in workflow
     if (workflowcfg.isTerminable() || workflowcfg.getTargetState() == TargetState.DELETE) {
       // clean up IS & EV
-      TaskUtil.cleanupIdealStateExtView(_manager.getHelixDataAccessor(), workflow);
+      cleanupIdealStateExtView(_manager.getHelixDataAccessor(), workflow);
 
       // delete workflow config
       PropertyKey workflowCfgKey = TaskUtil.getWorkflowConfigKey(accessor, workflow);
@@ -354,7 +435,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
 
     // Remove any idealstate and externalView.
-    TaskUtil.cleanupIdealStateExtView(accessor, job);
+    cleanupIdealStateExtView(accessor, job);
 
     // Remove DAG references in workflow
     PropertyKey workflowKey = TaskUtil.getWorkflowConfigKey(accessor, workflow);

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index 06b9751..9796497 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -32,6 +32,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.TestHelper;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskPartitionState;
 import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskUtil;
@@ -53,7 +54,7 @@ public class TaskTestUtil {
    * @param workflowResource Resource to poll for completeness
    * @throws InterruptedException
    */
-  public static void pollForWorkflowState(HelixManager manager, String workflowResource,
+  public static void pollForWorkflowState(TaskDriver driver, String workflowResource,
       TaskState... targetStates) throws InterruptedException {
     // Wait for completion.
     long st = System.currentTimeMillis();
@@ -61,7 +62,7 @@ public class TaskTestUtil {
     Set<TaskState> allowedStates = new HashSet<TaskState>(Arrays.asList(targetStates));
     do {
       Thread.sleep(100);
-      ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
+      ctx = driver.getWorkflowContext(workflowResource);
     } while ((ctx == null || ctx.getWorkflowState() == null || !allowedStates
         .contains(ctx.getWorkflowState())) && System.currentTimeMillis() < st + _default_timeout);
 
@@ -73,23 +74,23 @@ public class TaskTestUtil {
 
   /**
    * poll for job until it is at either state in targetStates.
-   * @param manager
+   * @param driver
    * @param workflowResource
    * @param jobName
    * @param targetStates
    * @throws InterruptedException
    */
-  public static void pollForJobState(HelixManager manager, String workflowResource, String jobName,
+  public static void pollForJobState(TaskDriver driver, String workflowResource, String jobName,
       TaskState... targetStates) throws InterruptedException {
     // Get workflow config
-    WorkflowConfig wfCfg = TaskUtil.getWorkflowCfg(manager, workflowResource);
+    WorkflowConfig wfCfg = driver.getWorkflowConfig(workflowResource);
     Assert.assertNotNull(wfCfg);
     WorkflowContext ctx;
     if (wfCfg.isRecurring()) {
       // if it's recurring, need to reconstruct workflow and job name
       do {
         Thread.sleep(100);
-        ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
+        ctx = driver.getWorkflowContext(workflowResource);
       } while ((ctx == null || ctx.getLastScheduledSingleWorkflow() == null));
       Assert.assertNotNull(ctx);
       Assert.assertNotNull(ctx.getLastScheduledSingleWorkflow());
@@ -103,7 +104,7 @@ public class TaskTestUtil {
     long st = System.currentTimeMillis();
     do {
       Thread.sleep(100);
-      ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
+      ctx = driver.getWorkflowContext(workflowResource);
     }
     while ((ctx == null || ctx.getJobState(jobName) == null || !allowedStates.contains(
         ctx.getJobState(jobName)))
@@ -114,27 +115,27 @@ public class TaskTestUtil {
         "expect job states: " + allowedStates + " actual job state: " + jobState);
   }
 
-  public static void pollForEmptyJobState(final HelixManager manager, final String workflowName,
+  public static void pollForEmptyJobState(final TaskDriver driver, final String workflowName,
       final String jobName) throws Exception {
     final String namespacedJobName = String.format("%s_%s", workflowName, jobName);
     boolean succeed = TestHelper.verify(new TestHelper.Verifier() {
 
       @Override
       public boolean verify() throws Exception {
-        WorkflowContext ctx = TaskUtil.getWorkflowContext(manager, workflowName);
+        WorkflowContext ctx = driver.getWorkflowContext(workflowName);
         return ctx == null || ctx.getJobState(namespacedJobName) == null;
       }
     }, _default_timeout);
     Assert.assertTrue(succeed);
   }
 
-  public static WorkflowContext pollForWorkflowContext(HelixManager manager, String workflowResource)
+  public static WorkflowContext pollForWorkflowContext(TaskDriver driver, String workflowResource)
       throws InterruptedException {
     // Wait for completion.
     long st = System.currentTimeMillis();
     WorkflowContext ctx;
     do {
-      ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
+      ctx = driver.getWorkflowContext(workflowResource);
       Thread.sleep(100);
     } while (ctx == null && System.currentTimeMillis() < st + _default_timeout);
     Assert.assertNotNull(ctx);
@@ -143,15 +144,15 @@ public class TaskTestUtil {
 
   // 1. Different jobs in a same work flow is in RUNNING at the same time
   // 2. No two jobs in the same work flow is in RUNNING at the same instance
-  public static boolean pollForWorkflowParallelState(HelixManager manager, String workflowName)
+  public static boolean pollForWorkflowParallelState(TaskDriver driver, String workflowName)
       throws InterruptedException {
 
-    WorkflowConfig workflowConfig = TaskUtil.getWorkflowCfg(manager, workflowName);
+    WorkflowConfig workflowConfig = driver.getWorkflowConfig(workflowName);
     Assert.assertNotNull(workflowConfig);
 
     WorkflowContext workflowContext = null;
     while (workflowContext == null) {
-      workflowContext = TaskUtil.getWorkflowContext(manager, workflowName);
+      workflowContext = driver.getWorkflowContext(workflowName);
       Thread.sleep(100);
     }
 
@@ -162,7 +163,7 @@ public class TaskTestUtil {
       finished = true;
       int runningCount = 0;
 
-      workflowContext = TaskUtil.getWorkflowContext(manager, workflowName);
+      workflowContext = driver.getWorkflowContext(workflowName);
       for (String jobName : workflowConfig.getJobDag().getAllNodes()) {
         TaskState jobState = workflowContext.getJobState(jobName);
         if (jobState == TaskState.IN_PROGRESS) {
@@ -177,9 +178,9 @@ public class TaskTestUtil {
 
       List<JobContext> jobContextList = new ArrayList<JobContext>();
       for (String jobName : workflowConfig.getJobDag().getAllNodes()) {
-        JobContext jobContext = TaskUtil.getJobContext(manager, jobName);
+        JobContext jobContext = driver.getJobContext(jobName);
         if (jobContext != null) {
-          jobContextList.add(TaskUtil.getJobContext(manager, jobName));
+          jobContextList.add(driver.getJobContext(jobName));
         }
       }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
index b23e268..f673f7b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
@@ -175,7 +175,7 @@ public class TestDisableJobExternalView extends ZkIntegrationTestBase {
 
     // ensure all jobs are completed
     String namedSpaceJob3 = String.format("%s_%s", queueName, "job3");
-    TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceJob3, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob3, TaskState.COMPLETED);
 
     Set<String> seenExternalViews = externviewChecker.getSeenExternalViews();
     String namedSpaceJob1 = String.format("%s_%s", queueName, "job1");

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
index ba8367e..1c58776 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
@@ -45,7 +45,6 @@ import org.apache.helix.task.TaskResult;
 import org.apache.helix.task.TaskResult.Status;
 import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.Workflow;
 import org.apache.helix.task.WorkflowContext;
 import org.apache.helix.tools.ClusterSetup;
@@ -157,8 +156,8 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
 
     // Ensure that each class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -184,8 +183,8 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
 
     // Ensure that each class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -213,8 +212,8 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
 
     // Ensure that each class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -241,8 +240,8 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
 
     // Ensure that the class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -276,13 +275,13 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
 
     // Ensure that the class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
 
     // Check that the workflow only started after the start time (with a 1 second buffer)
-    WorkflowContext workflowCtx = TaskUtil.getWorkflowContext(_manager, jobName);
+    WorkflowContext workflowCtx = _driver.getWorkflowContext(jobName);
     long startTime = workflowCtx.getStartTime();
     Assert.assertTrue((startTime + 1000) >= inFiveSeconds);
   }
@@ -308,10 +307,10 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure completion
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
 
     // Ensure a single retry happened
-    JobContext jobCtx = TaskUtil.getJobContext(_manager, jobName + "_" + jobName);
+    JobContext jobCtx = _driver.getJobContext(jobName + "_" + jobName);
     Assert.assertEquals(jobCtx.getPartitionNumAttempts(0), 2);
     Assert.assertTrue(jobCtx.getFinishTime() - jobCtx.getStartTime() >= delay);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index d83c5eb..ae3d52d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -179,13 +179,13 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
 
     _driver.start(queueBuild.build());
 
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
 
     // ensure job 1 is started before stop it
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
     String namedSpaceJob1 = String.format("%s_%s", scheduledQueue, currentJobNames.get(0));
     TaskTestUtil
-        .pollForJobState(_manager, scheduledQueue, namedSpaceJob1, TaskState.IN_PROGRESS);
+        .pollForJobState(_driver, scheduledQueue, namedSpaceJob1, TaskState.IN_PROGRESS);
 
     _driver.stop(queueName);
     _driver.delete(queueName);
@@ -208,18 +208,18 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
     _driver.createQueue(queueBuilder.build());
 
 
-    wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
 
     // ensure jobs are started and completed
     scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
     namedSpaceJob1 = String.format("%s_%s", scheduledQueue, currentJobNames.get(0));
     TaskTestUtil
-        .pollForJobState(_manager, scheduledQueue, namedSpaceJob1, TaskState.COMPLETED);
+        .pollForJobState(_driver, scheduledQueue, namedSpaceJob1, TaskState.COMPLETED);
 
     scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
     String namedSpaceJob2 = String.format("%s_%s", scheduledQueue, currentJobNames.get(1));
     TaskTestUtil
-        .pollForJobState(_manager, scheduledQueue, namedSpaceJob2, TaskState.COMPLETED);
+        .pollForJobState(_driver, scheduledQueue, namedSpaceJob2, TaskState.COMPLETED);
   }
 
   @Test
@@ -249,21 +249,21 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
     }
     _driver.createQueue(queueBuilder.build());
 
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
 
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", scheduledQueue, deletedJob1);
     TaskTestUtil
-        .pollForJobState(_manager, scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS,
+        .pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS,
             TaskState.COMPLETED);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + scheduledQueue);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.STOPPED);
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
 
     // delete the in-progress job (job 1) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob1);
@@ -274,21 +274,21 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
     _driver.resume(queueName);
 
     // ensure job 2 is started
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue,
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.IN_PROGRESS,
         TaskState.COMPLETED);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue,
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
     String deletedJob2 = currentJobNames.get(2);
     String namedSpaceDeletedJob2 = String.format("%s_%s", scheduledQueue, deletedJob2);
-    TaskTestUtil.pollForEmptyJobState(_manager, scheduledQueue, namedSpaceDeletedJob2);
+    TaskTestUtil.pollForEmptyJobState(_driver, scheduledQueue, namedSpaceDeletedJob2);
 
     // delete not-started job (job 3) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob2);
@@ -304,9 +304,9 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
     long preJobFinish = 0;
     for (int i = 0; i < currentJobNames.size(); i++) {
       String namedSpaceJobName = String.format("%s_%s", scheduledQueue, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
+      TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
 
-      JobContext jobContext = TaskUtil.getJobContext(_manager, namedSpaceJobName);
+      JobContext jobContext = _driver.getJobContext(namedSpaceJobName);
       long jobStart = jobContext.getStartTime();
       Assert.assertTrue(jobStart >= preJobFinish);
       preJobFinish = jobContext.getFinishTime();
@@ -349,12 +349,12 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
 
     String currentLastJob = jobNames.get(JOB_COUNTS - 2);
 
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
 
     // ensure all jobs are finished
     String namedSpaceJob = String.format("%s_%s", scheduledQueue, currentLastJob);
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
 
     // enqueue the last job
     LOG.info("Enqueuing job: " + jobNames.get(JOB_COUNTS - 1));
@@ -389,7 +389,7 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
 
     Assert.assertNull(accessor.getProperty(keyBuilder.idealStates(jobName)));
     Assert.assertNull(accessor.getProperty(keyBuilder.resourceConfig(jobName)));
-    TaskTestUtil.pollForEmptyJobState(_manager, queueName, jobName);
+    TaskTestUtil.pollForEmptyJobState(_driver, queueName, jobName);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index 9fd7735..101604b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -153,7 +153,8 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     List<String> currentJobNames = new ArrayList<String>();
     for (int i = 0; i < num_dbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(
+              _test_dbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
       String jobName = "job" + _test_dbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
@@ -164,8 +165,8 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(2));
-    TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceJob1, TaskState.FAILED);
-    TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.FAILED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
   }
 
   @Test
@@ -190,7 +191,7 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(0));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(0));
-    TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceJob1, TaskState.FAILED);
-    TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.FAILED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index 2d11f85..3a5b179 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -168,7 +168,7 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
             .setExpiry(expiry).build();
 
     _driver.start(flow);
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
 
     // Running workflow should have config and context viewable through accessor
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
@@ -182,7 +182,7 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     Assert.assertNotSame(accessor.getProperty(workflowCfgKey), null);
 
     // Wait for job to finish and expire
-    TaskTestUtil.pollForWorkflowState(_manager, jobName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
     Thread.sleep(expiry);
     TaskUtil.invokeRebalance(_manager.getHelixDataAccessor(), flow.getName());
     Thread.sleep(expiry);
@@ -212,10 +212,10 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(flow);
 
     // Wait for job completion
-    TaskTestUtil.pollForWorkflowState(_manager, jobResource, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
 
     // Ensure all partitions are completed individually
-    JobContext ctx = TaskUtil.getJobContext(_manager, TaskUtil.getNamespacedJobName(jobResource));
+    JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     for (int i = 0; i < NUM_PARTITIONS; i++) {
       Assert.assertEquals(ctx.getPartitionState(i), TaskPartitionState.COMPLETED);
       Assert.assertEquals(ctx.getPartitionNumAttempts(i), 1);
@@ -239,13 +239,13 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(flow);
 
     // wait for job completeness/timeout
-    TaskTestUtil.pollForWorkflowState(_manager, jobResource, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
 
     // see if resulting context completed successfully for our partition set
     String namespacedName = TaskUtil.getNamespacedJobName(jobResource);
 
-    JobContext ctx = TaskUtil.getJobContext(_manager, namespacedName);
-    WorkflowContext workflowContext = TaskUtil.getWorkflowContext(_manager, jobResource);
+    JobContext ctx = _driver.getJobContext(namespacedName);
+    WorkflowContext workflowContext = _driver.getWorkflowContext(jobResource);
     Assert.assertNotNull(ctx);
     Assert.assertNotNull(workflowContext);
     Assert.assertEquals(workflowContext.getJobState(namespacedName), TaskState.COMPLETED);
@@ -264,11 +264,11 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     new TaskDriver(_manager).start(flow);
 
     // Wait until the workflow completes
-    TaskTestUtil.pollForWorkflowState(_manager, workflowName, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, workflowName, TaskState.COMPLETED);
 
     // Assert completion for all tasks within two minutes
     for (String task : flow.getJobConfigs().keySet()) {
-      TaskTestUtil.pollForJobState(_manager, workflowName, task, TaskState.COMPLETED);
+      TaskTestUtil.pollForJobState(_driver, workflowName, task, TaskState.COMPLETED);
     }
   }
 
@@ -284,10 +284,10 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     _driver.start(flow);
 
     // Wait until the job reports failure.
-    TaskTestUtil.pollForWorkflowState(_manager, jobResource, TaskState.FAILED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.FAILED);
 
     // Check that all partitions timed out up to maxAttempts
-    JobContext ctx = TaskUtil.getJobContext(_manager, TaskUtil.getNamespacedJobName(jobResource));
+    JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     int maxAttempts = 0;
     for (int i = 0; i < NUM_PARTITIONS; i++) {
       TaskPartitionState state = ctx.getPartitionState(i);
@@ -322,10 +322,10 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     // Ensure successful completion
     String namespacedJob1 = queueName + "_masterJob";
     String namespacedJob2 = queueName + "_slaveJob";
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob1, TaskState.COMPLETED);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob2, TaskState.COMPLETED);
-    JobContext masterJobContext = TaskUtil.getJobContext(_manager, namespacedJob1);
-    JobContext slaveJobContext = TaskUtil.getJobContext(_manager, namespacedJob2);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
+    JobContext masterJobContext = _driver.getJobContext(namespacedJob1);
+    JobContext slaveJobContext = _driver.getJobContext(namespacedJob2);
 
     // Ensure correct ordering
     long job1Finish = masterJobContext.getFinishTime();
@@ -340,7 +340,7 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     Assert.assertNull(accessor.getProperty(keyBuilder.resourceConfig(namespacedJob1)));
     Assert.assertNull(accessor.getProperty(keyBuilder.idealStates(namespacedJob2)));
     Assert.assertNull(accessor.getProperty(keyBuilder.resourceConfig(namespacedJob2)));
-    WorkflowConfig workflowCfg = TaskUtil.getWorkflowCfg(_manager, queueName);
+    WorkflowConfig workflowCfg = _driver.getWorkflowConfig(queueName);
     JobDag dag = workflowCfg.getJobDag();
     Assert.assertFalse(dag.getAllNodes().contains(namespacedJob1));
     Assert.assertFalse(dag.getAllNodes().contains(namespacedJob2));

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
index a778dcd..8051b2f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
@@ -151,13 +151,13 @@ public class TestTaskRebalancerFailover extends ZkUnitTestBase {
 
     // check all tasks completed on MASTER
     String namespacedJob1 = String.format("%s_%s", queueName, job1Name);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob1, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
 
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
     ExternalView ev =
         accessor.getProperty(keyBuilder.externalView(WorkflowGenerator.DEFAULT_TGT_DB));
-    JobContext ctx = TaskUtil.getJobContext(_manager, namespacedJob1);
+    JobContext ctx = _driver.getJobContext(namespacedJob1);
     Set<String> failOverPartitions = Sets.newHashSet();
     for (int p = 0; p < _p; p++) {
       String instanceName = ctx.getAssignedParticipant(p);
@@ -178,12 +178,12 @@ public class TestTaskRebalancerFailover extends ZkUnitTestBase {
     LOG.info("Enqueuing job: " + job2Name);
     _driver.enqueueJob(queueName, job2Name, job);
 
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob2, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.IN_PROGRESS);
     _participants[0].syncStop();
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob2, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
 
     // tasks previously assigned to localhost_12918 should be re-scheduled on new master
-    ctx = TaskUtil.getJobContext(_manager, namespacedJob2);
+    ctx = _driver.getJobContext(namespacedJob2);
     ev = accessor.getProperty(keyBuilder.externalView(WorkflowGenerator.DEFAULT_TGT_DB));
     for (int p = 0; p < _p; p++) {
       String partitionName = ctx.getTargetForPartition(p);
@@ -204,7 +204,7 @@ public class TestTaskRebalancerFailover extends ZkUnitTestBase {
     Assert.assertNull(accessor.getProperty(keyBuilder.resourceConfig(namespacedJob1)));
     Assert.assertNull(accessor.getProperty(keyBuilder.idealStates(namespacedJob2)));
     Assert.assertNull(accessor.getProperty(keyBuilder.resourceConfig(namespacedJob2)));
-    WorkflowConfig workflowCfg = TaskUtil.getWorkflowCfg(_manager, queueName);
+    WorkflowConfig workflowCfg = _driver.getWorkflowConfig(queueName);
     JobDag dag = workflowCfg.getJobDag();
     Assert.assertFalse(dag.getAllNodes().contains(namespacedJob1));
     Assert.assertFalse(dag.getAllNodes().contains(namespacedJob2));

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
index c9a0445..580f5ac 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
@@ -158,6 +158,6 @@ public class TestTaskRebalancerParallel extends ZkIntegrationTestBase {
       _driver.enqueueJob(queueName, "job_" + (i + 1), jobConfigBuilders.get(i));
     }
 
-    Assert.assertTrue(TaskTestUtil.pollForWorkflowParallelState(_manager, queueName));
+    Assert.assertTrue(TaskTestUtil.pollForWorkflowParallelState(_driver, queueName));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
index 8fec899..e576304 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
@@ -138,9 +138,9 @@ public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
     _driver.start(flow);
 
     // Wait until the job completes.
-    TaskTestUtil.pollForWorkflowState(_manager, jobResource, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
 
-    JobContext ctx = TaskUtil.getJobContext(_manager, TaskUtil.getNamespacedJobName(jobResource));
+    JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     for (int i = 0; i < _p; i++) {
       TaskPartitionState state = ctx.getPartitionState(i);
       if (state != null) {

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
index 7a8d305..30cb460 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
@@ -168,15 +168,15 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     LOG.info("Starting flow " + flow.getName());
     _driver.start(flow);
-    TaskTestUtil.pollForWorkflowState(_manager, JOB_RESOURCE, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, JOB_RESOURCE, TaskState.IN_PROGRESS);
 
     LOG.info("Pausing job");
     _driver.stop(JOB_RESOURCE);
-    TaskTestUtil.pollForWorkflowState(_manager, JOB_RESOURCE, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, JOB_RESOURCE, TaskState.STOPPED);
 
     LOG.info("Resuming job");
     _driver.resume(JOB_RESOURCE);
-    TaskTestUtil.pollForWorkflowState(_manager, JOB_RESOURCE, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, JOB_RESOURCE, TaskState.COMPLETED);
   }
 
   @Test
@@ -186,15 +186,15 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     LOG.info("Starting flow " + workflow);
     _driver.start(flow);
-    TaskTestUtil.pollForWorkflowState(_manager, workflow, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, workflow, TaskState.IN_PROGRESS);
 
     LOG.info("Pausing workflow");
     _driver.stop(workflow);
-    TaskTestUtil.pollForWorkflowState(_manager, workflow, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, workflow, TaskState.STOPPED);
 
     LOG.info("Resuming workflow");
     _driver.resume(workflow);
-    TaskTestUtil.pollForWorkflowState(_manager, workflow, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, workflow, TaskState.COMPLETED);
   }
 
   @Test
@@ -224,27 +224,27 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     _driver.enqueueJob(queueName, job2Name, job2);
 
     String namespacedJob1 = String.format("%s_%s", queueName,  job1Name);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob1, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.IN_PROGRESS);
 
     // stop job1
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.STOPPED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.STOPPED);
 
     // Ensure job2 is not started
     TimeUnit.MILLISECONDS.sleep(200);
     String namespacedJob2 = String.format("%s_%s", queueName, job2Name);
-    TaskTestUtil.pollForEmptyJobState(_manager, queueName, job2Name);
+    TaskTestUtil.pollForEmptyJobState(_driver, queueName, job2Name);
 
     LOG.info("Resuming job-queue: " + queueName);
     _driver.resume(queueName);
 
     // Ensure successful completion
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob1, TaskState.COMPLETED);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob2, TaskState.COMPLETED);
-    JobContext masterJobContext = TaskUtil.getJobContext(_manager, namespacedJob1);
-    JobContext slaveJobContext = TaskUtil.getJobContext(_manager, namespacedJob2);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
+    JobContext masterJobContext = _driver.getJobContext(namespacedJob1);
+    JobContext slaveJobContext = _driver.getJobContext(namespacedJob2);
 
     // Ensure correct ordering
     long job1Finish = masterJobContext.getFinishTime();
@@ -290,13 +290,13 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", queueName, deletedJob1);
-    TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceDeletedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.STOPPED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceDeletedJob1, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.STOPPED);
 
     // delete the in-progress job (job 1) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob1);
@@ -307,19 +307,19 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     // ensure job 2 is started
     TaskTestUtil
-        .pollForJobState(_manager, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.IN_PROGRESS);
+        .pollForJobState(_driver, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
     TaskTestUtil
-        .pollForJobState(_manager, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.STOPPED);
+        .pollForJobState(_driver, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
     String deletedJob2 = currentJobNames.get(2);
     String namedSpaceDeletedJob2 = String.format("%s_%s", queueName, deletedJob2);
-    TaskTestUtil.pollForEmptyJobState(_manager, queueName, namedSpaceDeletedJob2);
+    TaskTestUtil.pollForEmptyJobState(_driver, queueName, namedSpaceDeletedJob2);
 
     // delete not-started job (job 3) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob2);
@@ -343,9 +343,9 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     long preJobFinish = 0;
     for (int i = 0; i < currentJobNames.size(); i++) {
       String namedSpaceJobName = String.format("%s_%s", queueName, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceJobName, TaskState.COMPLETED);
+      TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJobName, TaskState.COMPLETED);
 
-      JobContext jobContext = TaskUtil.getJobContext(_manager, namedSpaceJobName);
+      JobContext jobContext = _driver.getJobContext(namedSpaceJobName);
       long jobStart = jobContext.getStartTime();
       Assert.assertTrue(jobStart >= preJobFinish);
       preJobFinish = jobContext.getFinishTime();
@@ -391,20 +391,20 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     _driver.createQueue(queueBuilder.build());
 
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
 
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", scheduledQueue, deletedJob1);
     TaskTestUtil
-        .pollForJobState(_manager, scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
+        .pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + scheduledQueue);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.STOPPED);
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
 
     // delete the in-progress job (job 1) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob1);
@@ -415,20 +415,20 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     _driver.resume(queueName);
 
     // ensure job 2 is started
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue,
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue,
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.STOPPED);
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
     String deletedJob2 = currentJobNames.get(2);
     String namedSpaceDeletedJob2 = String.format("%s_%s", scheduledQueue, deletedJob2);
-    TaskTestUtil.pollForEmptyJobState(_manager, scheduledQueue, namedSpaceDeletedJob2);
+    TaskTestUtil.pollForEmptyJobState(_driver, scheduledQueue, namedSpaceDeletedJob2);
 
     // delete not-started job (job 3) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob2);
@@ -444,9 +444,9 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     long preJobFinish = 0;
     for (int i = 0; i < currentJobNames.size(); i++) {
       String namedSpaceJobName = String.format("%s_%s", scheduledQueue, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
+      TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
 
-      JobContext jobContext = TaskUtil.getJobContext(_manager, namedSpaceJobName);
+      JobContext jobContext = _driver.getJobContext(namedSpaceJobName);
       long jobStart = jobContext.getStartTime();
       Assert.assertTrue(jobStart >= preJobFinish);
       preJobFinish = jobContext.getFinishTime();
@@ -487,12 +487,12 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     String currentLastJob = jobNames.get(JOB_COUNTS - 2);
 
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
 
     // ensure all jobs are finished
     String namedSpaceJob = String.format("%s_%s", scheduledQueue, currentLastJob);
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
 
     // enqueue the last job
     LOG.info("Enqueuing job: " + jobNames.get(JOB_COUNTS - 1));
@@ -535,10 +535,10 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     _driver.enqueueJob(queueName, job2Name, job2);
 
     String namespacedJob1 = String.format("%s_%s", queueName,  job1Name);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob1, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
 
     String namespacedJob2 = String.format("%s_%s", queueName,  job2Name);
-    TaskTestUtil.pollForJobState(_manager, queueName, namespacedJob2, TaskState.COMPLETED);
+    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
 
     // Stop queue
     _driver.stop(queueName);
@@ -588,11 +588,11 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
     Assert.assertNull(accessor.getProperty(keyBuilder.idealStates(jobName)));
     Assert.assertNull(accessor.getProperty(keyBuilder.resourceConfig(jobName)));
-    TaskTestUtil.pollForEmptyJobState(_manager, queueName, jobName);
+    TaskTestUtil.pollForEmptyJobState(_driver, queueName, jobName);
   }
 
   private void verifyJobNotInQueue(String queueName, String namedSpacedJobName) {
-    WorkflowConfig workflowCfg = TaskUtil.getWorkflowCfg(_manager, queueName);
+    WorkflowConfig workflowCfg = _driver.getWorkflowConfig(queueName);
     JobDag dag = workflowCfg.getJobDag();
     Assert.assertFalse(dag.getAllNodes().contains(namedSpacedJobName));
     Assert.assertFalse(dag.getChildrenToParents().containsKey(namedSpacedJobName));

http://git-wip-us.apache.org/repos/asf/helix/blob/579d82fd/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
index fc93392..964f9e1 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
@@ -19,13 +19,10 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
-import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
-import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.integration.ZkIntegrationTestBase;
@@ -35,7 +32,6 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
-import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.ScheduleConfig;
 import org.apache.helix.task.Task;
@@ -44,8 +40,6 @@ import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.TaskUtil;
-import org.apache.helix.task.Workflow;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
 import org.apache.helix.tools.ClusterSetup;
@@ -58,7 +52,6 @@ import org.testng.annotations.Test;
 
 import java.util.ArrayList;
 import java.util.Calendar;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -180,9 +173,9 @@ public class TestUpdateWorkflow extends ZkIntegrationTestBase {
 
     _driver.start(queueBuild.build());
 
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
 
-    WorkflowConfig workflowConfig = TaskUtil.getWorkflowCfg(_manager, queueName);
+    WorkflowConfig workflowConfig = _driver.getWorkflowConfig(queueName);
     WorkflowConfig.Builder configBuilder = new WorkflowConfig.Builder(workflowConfig);
 
     Calendar startTime = Calendar.getInstance();
@@ -195,18 +188,18 @@ public class TestUpdateWorkflow extends ZkIntegrationTestBase {
 
     // ensure current schedule is started
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.IN_PROGRESS);
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.IN_PROGRESS);
 
     _driver.updateWorkflow(queueName, configBuilder.build());
 
     // ensure current schedule is completed
-    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.COMPLETED);
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.COMPLETED);
 
     Thread.sleep(1000);
 
-    wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
     scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-    WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_manager, scheduledQueue);
+    WorkflowConfig wCfg = _driver.getWorkflowConfig(scheduledQueue);
 
     Calendar configStartTime = Calendar.getInstance();
     configStartTime.setTime(wCfg.getStartTime());


[11/33] helix git commit: Refactor Workflow and Jobqueue builders to make the builder API more clean.

Posted by lx...@apache.org.
Refactor Workflow and Jobqueue builders to make the builder API more clean.


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

Branch: refs/heads/helix-0.6.x
Commit: d386aff394f2e4e7202f13fe2ed5e6533a8cfb29
Parents: 66dba1f
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Feb 23 17:06:35 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:48:35 2016 -0700

----------------------------------------------------------------------
 .../webapp/resources/JobQueuesResource.java     |   8 +-
 .../webapp/resources/WorkflowsResource.java     |   4 +-
 .../helix/task/DeprecatedTaskRebalancer.java    |  21 +-
 .../java/org/apache/helix/task/JobConfig.java   | 144 ++++++-------
 .../main/java/org/apache/helix/task/JobDag.java |   5 +-
 .../java/org/apache/helix/task/JobQueue.java    |  98 +++++----
 .../java/org/apache/helix/task/TaskDriver.java  |  36 ++--
 .../java/org/apache/helix/task/TaskUtil.java    |  35 ----
 .../java/org/apache/helix/task/Workflow.java    | 156 ++++++++------
 .../org/apache/helix/task/WorkflowConfig.java   | 207 +++++++++++++++----
 .../apache/helix/task/WorkflowRebalancer.java   |  43 ++--
 .../helix/integration/task/TaskTestUtil.java    |  48 +++--
 .../integration/task/TestRecurringJobQueue.java |  54 ++++-
 .../task/TestTaskRebalancerParallel.java        |  11 +-
 .../integration/task/TestUpdateWorkflow.java    |  99 +++++++--
 .../integration/task/WorkflowGenerator.java     |   9 -
 16 files changed, 605 insertions(+), 373 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueuesResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueuesResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueuesResource.java
index 1a5cb17..e0a0657 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueuesResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueuesResource.java
@@ -103,9 +103,11 @@ public class JobQueuesResource extends ServerResource {
       Map.Entry<String, HelixProperty> e = it.next();
       HelixProperty resource = e.getValue();
       Map<String, String> simpleFields = resource.getRecord().getSimpleFields();
-      boolean isTerminable = resource.getRecord().getBooleanField(WorkflowConfig.TERMINABLE, true);
-      if (!simpleFields.containsKey(WorkflowConfig.TARGET_STATE)
-          || !simpleFields.containsKey(WorkflowConfig.DAG) || isTerminable) {
+      boolean isTerminable = resource.getRecord()
+          .getBooleanField(WorkflowConfig.WorkflowConfigProperty.Terminable.name(), true);
+      if (!simpleFields.containsKey(WorkflowConfig.WorkflowConfigProperty.TargetState.name())
+          || !simpleFields.containsKey(WorkflowConfig.WorkflowConfigProperty.Dag.name())
+          || isTerminable) {
         it.remove();
       }
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/WorkflowsResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/WorkflowsResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/WorkflowsResource.java
index 9175530..c517fea 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/WorkflowsResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/WorkflowsResource.java
@@ -96,8 +96,8 @@ public class WorkflowsResource extends ServerResource {
       Map.Entry<String, HelixProperty> e = it.next();
       HelixProperty resource = e.getValue();
       Map<String, String> simpleFields = resource.getRecord().getSimpleFields();
-      if (!simpleFields.containsKey(WorkflowConfig.TARGET_STATE)
-          || !simpleFields.containsKey(WorkflowConfig.DAG)) {
+      if (!simpleFields.containsKey(WorkflowConfig.WorkflowConfigProperty.TargetState.name())
+          || !simpleFields.containsKey(WorkflowConfig.WorkflowConfigProperty.Dag.name())) {
         it.remove();
       }
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
index 6f744f0..855312b 100644
--- a/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
@@ -673,19 +673,21 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
     // Create a new workflow with a new name
     HelixProperty workflowConfig = resourceConfigMap.get(origWorkflowName);
     Map<String, String> wfSimpleFields = workflowConfig.getRecord().getSimpleFields();
-    JobDag jobDag = JobDag.fromJson(wfSimpleFields.get(WorkflowConfig.DAG));
+    JobDag jobDag =
+        JobDag.fromJson(wfSimpleFields.get(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
     Map<String, Set<String>> parentsToChildren = jobDag.getParentsToChildren();
     Workflow.Builder builder = new Workflow.Builder(newWorkflowName);
 
     // Set the workflow expiry
-    builder.setExpiry(Long.parseLong(wfSimpleFields.get(WorkflowConfig.EXPIRY)));
+    builder.setExpiry(
+        Long.parseLong(wfSimpleFields.get(WorkflowConfig.WorkflowConfigProperty.Expiry.name())));
 
     // Set the schedule, if applicable
     ScheduleConfig scheduleConfig;
     if (newStartTime != null) {
       scheduleConfig = ScheduleConfig.oneTimeDelayedStart(newStartTime);
     } else {
-      scheduleConfig = TaskUtil.parseScheduleFromConfigMap(wfSimpleFields);
+      scheduleConfig = WorkflowConfig.parseScheduleFromConfigMap(wfSimpleFields);
     }
     if (scheduleConfig != null) {
       builder.setScheduleConfig(scheduleConfig);
@@ -699,7 +701,7 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
         String job = TaskUtil.getDenamespacedJobName(origWorkflowName, namespacedJob);
         HelixProperty jobConfig = resourceConfigMap.get(namespacedJob);
         Map<String, String> jobSimpleFields = jobConfig.getRecord().getSimpleFields();
-        jobSimpleFields.put(JobConfig.JobConfigProperty.WORKFLOW_ID.value(), newWorkflowName); // overwrite workflow name
+        jobSimpleFields.put(JobConfig.JobConfigProperty.WorkflowID.name(), newWorkflowName); // overwrite workflow name
         for (Map.Entry<String, String> e : jobSimpleFields.entrySet()) {
           builder.addConfig(job, e.getKey(), e.getValue());
         }
@@ -746,7 +748,8 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
     long currentTime = now;
     Date scheduledTime = SCHEDULED_TIMES.get(jobResource);
     if (scheduledTime != null && currentTime > scheduledTime.getTime()) {
-      LOG.debug("Remove schedule timer for" + jobResource + " time: " + SCHEDULED_TIMES.get(jobResource));
+      LOG.debug(
+          "Remove schedule timer for" + jobResource + " time: " + SCHEDULED_TIMES.get(jobResource));
       SCHEDULED_TIMES.remove(jobResource);
     }
 
@@ -831,7 +834,7 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
   private static void markForDeletion(HelixManager mgr, String resourceName) {
     mgr.getConfigAccessor().set(
         TaskUtil.getResourceConfigScope(mgr.getClusterName(), resourceName),
-        WorkflowConfig.TARGET_STATE, TargetState.DELETE.name());
+        WorkflowConfig.WorkflowConfigProperty.TargetState.name(), TargetState.DELETE.name());
   }
 
   /**
@@ -848,7 +851,8 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
     DataUpdater<ZNRecord> dagRemover = new DataUpdater<ZNRecord>() {
       @Override
       public ZNRecord update(ZNRecord currentData) {
-        JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
+        JobDag jobDag = JobDag
+            .fromJson(currentData.getSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
         for (String child : jobDag.getDirectChildren(resourceName)) {
           jobDag.getChildrenToParents().get(child).remove(resourceName);
         }
@@ -859,7 +863,8 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
         jobDag.getParentsToChildren().remove(resourceName);
         jobDag.getAllNodes().remove(resourceName);
         try {
-          currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
+          currentData
+              .setSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name(), jobDag.toJson());
         } catch (Exception e) {
           LOG.equals("Could not update DAG for job: " + resourceName);
         }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index 65a9caf..d423d38 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -44,81 +44,71 @@ public class JobConfig {
     /**
      * The name of the workflow to which the job belongs.
      */
-    WORKFLOW_ID("WorkflowID"),
+    WorkflowID,
     /**
      * The assignment strategy of this job
      */
-    ASSIGNMENT_STRATEGY("AssignmentStrategy"),
+    AssignmentStrategy,
     /**
      * The name of the target resource.
      */
-    TARGET_RESOURCE("TargetResource"),
+    TargetResource,
     /**
      * The set of the target partition states. The value must be a comma-separated list of partition
      * states.
      */
-    TARGET_PARTITION_STATES("TargetPartitionStates"),
+    TargetPartitionStates,
     /**
      * The set of the target partition ids. The value must be a comma-separated list of partition ids.
      */
-    TARGET_PARTITIONS("TargetPartitions"),
+    TargetPartitions,
     /**
      * The command that is to be run by participants in the case of identical tasks.
      */
-    COMMAND("Command"),
+    Command,
     /**
      * The command configuration to be used by the tasks.
      */
-    JOB_COMMAND_CONFIG_MAP("JobCommandConfig"),
+    JobCommandConfig,
     /**
      * The timeout for a task.
      */
-    TIMEOUT_PER_TASK("TimeoutPerPartition"),
+    TimeoutPerPartition,
     /**
      * The maximum number of times the task rebalancer may attempt to execute a task.
      */
-    MAX_ATTEMPTS_PER_TASK("MaxAttemptsPerTask"),
+    MaxAttemptsPerTask,
     /**
      * The maximum number of times Helix will intentionally move a failing task
      */
-    MAX_FORCED_REASSIGNMENTS_PER_TASK("MaxForcedReassignmentsPerTask"),
+    MaxForcedReassignmentsPerTask,
     /**
      * The number of concurrent tasks that are allowed to run on an instance.
      */
-    NUM_CONCURRENT_TASKS_PER_INSTANCE("ConcurrentTasksPerInstance"),
+    ConcurrentTasksPerInstance,
     /**
      * The number of tasks within the job that are allowed to fail.
      */
-    FAILURE_THRESHOLD("FailureThreshold"),
+    FailureThreshold,
     /**
      * The amount of time in ms to wait before retrying a task
      */
-    TASK_RETRY_DELAY("TaskRetryDelay"),
+    TaskRetryDelay,
 
     /**
      * Whether failure of directly dependent jobs should fail this job.
      */
-    IGNORE_DEPENDENT_JOB_FAILURE("IgnoreDependentJobFailure"),
+    IgnoreDependentJobFailure,
 
     /**
      * The individual task configurations, if any *
      */
-    TASK_CONFIGS("TaskConfigs"),
+    TaskConfigs,
 
     /**
      * Disable external view (not showing) for this job resource
      */
-    DISABLE_EXTERNALVIEW("DisableExternalView");
-
-    private final String _value;
-
-    private JobConfigProperty(String val) {
-      _value = val;
-    }
-
-    public String value() {
-      return _value;
-    }
+    DisableExternalView
   }
 
   //Default property values
@@ -237,40 +227,40 @@ public class JobConfig {
 
   public Map<String, String> getResourceConfigMap() {
     Map<String, String> cfgMap = new HashMap<String, String>();
-    cfgMap.put(JobConfigProperty.WORKFLOW_ID.value(), _workflow);
+    cfgMap.put(JobConfigProperty.WorkflowID.name(), _workflow);
     if (_command != null) {
-      cfgMap.put(JobConfigProperty.COMMAND.value(), _command);
+      cfgMap.put(JobConfigProperty.Command.name(), _command);
     }
     if (_jobCommandConfigMap != null) {
       String serializedConfig = TaskUtil.serializeJobCommandConfigMap(_jobCommandConfigMap);
       if (serializedConfig != null) {
-        cfgMap.put(JobConfigProperty.JOB_COMMAND_CONFIG_MAP.value(), serializedConfig);
+        cfgMap.put(JobConfigProperty.JobCommandConfig.name(), serializedConfig);
       }
     }
     if (_targetResource != null) {
-      cfgMap.put(JobConfigProperty.TARGET_RESOURCE.value(), _targetResource);
+      cfgMap.put(JobConfigProperty.TargetResource.name(), _targetResource);
     }
     if (_targetPartitionStates != null) {
-      cfgMap.put(JobConfigProperty.TARGET_PARTITION_STATES.value(),
+      cfgMap.put(JobConfigProperty.TargetPartitionStates.name(),
           Joiner.on(",").join(_targetPartitionStates));
     }
     if (_targetPartitions != null) {
       cfgMap
-          .put(JobConfigProperty.TARGET_PARTITIONS.value(), Joiner.on(",").join(_targetPartitions));
+          .put(JobConfigProperty.TargetPartitions.name(), Joiner.on(",").join(_targetPartitions));
     }
     if (_retryDelay > 0) {
-      cfgMap.put(JobConfigProperty.TASK_RETRY_DELAY.value(), "" + _retryDelay);
+      cfgMap.put(JobConfigProperty.TaskRetryDelay.name(), "" + _retryDelay);
     }
-    cfgMap.put(JobConfigProperty.TIMEOUT_PER_TASK.value(), "" + _timeoutPerTask);
-    cfgMap.put(JobConfigProperty.MAX_ATTEMPTS_PER_TASK.value(), "" + _maxAttemptsPerTask);
-    cfgMap.put(JobConfigProperty.MAX_FORCED_REASSIGNMENTS_PER_TASK.value(),
+    cfgMap.put(JobConfigProperty.TimeoutPerPartition.name(), "" + _timeoutPerTask);
+    cfgMap.put(JobConfigProperty.MaxAttemptsPerTask.name(), "" + _maxAttemptsPerTask);
+    cfgMap.put(JobConfigProperty.MaxForcedReassignmentsPerTask.name(),
         "" + _maxForcedReassignmentsPerTask);
-    cfgMap.put(JobConfigProperty.FAILURE_THRESHOLD.value(), "" + _failureThreshold);
-    cfgMap.put(JobConfigProperty.DISABLE_EXTERNALVIEW.value(),
+    cfgMap.put(JobConfigProperty.FailureThreshold.name(), "" + _failureThreshold);
+    cfgMap.put(JobConfigProperty.DisableExternalView.name(),
         Boolean.toString(_disableExternalView));
-    cfgMap.put(JobConfigProperty.NUM_CONCURRENT_TASKS_PER_INSTANCE.value(),
+    cfgMap.put(JobConfigProperty.ConcurrentTasksPerInstance.name(),
         "" + _numConcurrentTasksPerInstance);
-    cfgMap.put(JobConfigProperty.IGNORE_DEPENDENT_JOB_FAILURE.value(),
+    cfgMap.put(JobConfigProperty.IgnoreDependentJobFailure.name(),
         Boolean.toString(_ignoreDependentJobFailure));
     return cfgMap;
   }
@@ -312,56 +302,56 @@ public class JobConfig {
      */
     public static Builder fromMap(Map<String, String> cfg) {
       Builder b = new Builder();
-      if (cfg.containsKey(JobConfigProperty.WORKFLOW_ID.value())) {
-        b.setWorkflow(cfg.get(JobConfigProperty.WORKFLOW_ID.value()));
+      if (cfg.containsKey(JobConfigProperty.WorkflowID.name())) {
+        b.setWorkflow(cfg.get(JobConfigProperty.WorkflowID.name()));
       }
-      if (cfg.containsKey(JobConfigProperty.TARGET_RESOURCE.value())) {
-        b.setTargetResource(cfg.get(JobConfigProperty.TARGET_RESOURCE.value()));
+      if (cfg.containsKey(JobConfigProperty.TargetResource.name())) {
+        b.setTargetResource(cfg.get(JobConfigProperty.TargetResource.name()));
       }
-      if (cfg.containsKey(JobConfigProperty.TARGET_PARTITIONS.value())) {
-        b.setTargetPartitions(csvToStringList(cfg.get(JobConfigProperty.TARGET_PARTITIONS.value())));
+      if (cfg.containsKey(JobConfigProperty.TargetPartitions.name())) {
+        b.setTargetPartitions(csvToStringList(cfg.get(JobConfigProperty.TargetPartitions.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.TARGET_PARTITION_STATES.value())) {
+      if (cfg.containsKey(JobConfigProperty.TargetPartitionStates.name())) {
         b.setTargetPartitionStates(new HashSet<String>(
-            Arrays.asList(cfg.get(JobConfigProperty.TARGET_PARTITION_STATES.value()).split(","))));
+            Arrays.asList(cfg.get(JobConfigProperty.TargetPartitionStates.name()).split(","))));
       }
-      if (cfg.containsKey(JobConfigProperty.COMMAND.value())) {
-        b.setCommand(cfg.get(JobConfigProperty.COMMAND.value()));
+      if (cfg.containsKey(JobConfigProperty.Command.name())) {
+        b.setCommand(cfg.get(JobConfigProperty.Command.name()));
       }
-      if (cfg.containsKey(JobConfigProperty.JOB_COMMAND_CONFIG_MAP.value())) {
+      if (cfg.containsKey(JobConfigProperty.JobCommandConfig.name())) {
         Map<String, String> commandConfigMap = TaskUtil.deserializeJobCommandConfigMap(
-            cfg.get(JobConfigProperty.JOB_COMMAND_CONFIG_MAP.value()));
+            cfg.get(JobConfigProperty.JobCommandConfig.name()));
         b.setJobCommandConfigMap(commandConfigMap);
       }
-      if (cfg.containsKey(JobConfigProperty.TIMEOUT_PER_TASK.value())) {
-        b.setTimeoutPerTask(Long.parseLong(cfg.get(JobConfigProperty.TIMEOUT_PER_TASK.value())));
+      if (cfg.containsKey(JobConfigProperty.TimeoutPerPartition.name())) {
+        b.setTimeoutPerTask(Long.parseLong(cfg.get(JobConfigProperty.TimeoutPerPartition.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.NUM_CONCURRENT_TASKS_PER_INSTANCE.value())) {
+      if (cfg.containsKey(JobConfigProperty.ConcurrentTasksPerInstance.name())) {
         b.setNumConcurrentTasksPerInstance(
-            Integer.parseInt(cfg.get(JobConfigProperty.NUM_CONCURRENT_TASKS_PER_INSTANCE.value())));
+            Integer.parseInt(cfg.get(JobConfigProperty.ConcurrentTasksPerInstance.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.MAX_ATTEMPTS_PER_TASK.value())) {
+      if (cfg.containsKey(JobConfigProperty.MaxAttemptsPerTask.name())) {
         b.setMaxAttemptsPerTask(
-            Integer.parseInt(cfg.get(JobConfigProperty.MAX_ATTEMPTS_PER_TASK.value())));
+            Integer.parseInt(cfg.get(JobConfigProperty.MaxAttemptsPerTask.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.MAX_FORCED_REASSIGNMENTS_PER_TASK.value())) {
+      if (cfg.containsKey(JobConfigProperty.MaxForcedReassignmentsPerTask.name())) {
         b.setMaxForcedReassignmentsPerTask(
-            Integer.parseInt(cfg.get(JobConfigProperty.MAX_FORCED_REASSIGNMENTS_PER_TASK.value())));
+            Integer.parseInt(cfg.get(JobConfigProperty.MaxForcedReassignmentsPerTask.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.FAILURE_THRESHOLD.value())) {
+      if (cfg.containsKey(JobConfigProperty.FailureThreshold.name())) {
         b.setFailureThreshold(
-            Integer.parseInt(cfg.get(JobConfigProperty.FAILURE_THRESHOLD.value())));
+            Integer.parseInt(cfg.get(JobConfigProperty.FailureThreshold.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.TASK_RETRY_DELAY.value())) {
-        b.setTaskRetryDelay(Long.parseLong(cfg.get(JobConfigProperty.TASK_RETRY_DELAY.value())));
+      if (cfg.containsKey(JobConfigProperty.TaskRetryDelay.name())) {
+        b.setTaskRetryDelay(Long.parseLong(cfg.get(JobConfigProperty.TaskRetryDelay.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.DISABLE_EXTERNALVIEW.value())) {
+      if (cfg.containsKey(JobConfigProperty.DisableExternalView.name())) {
         b.setDisableExternalView(
-            Boolean.valueOf(cfg.get(JobConfigProperty.DISABLE_EXTERNALVIEW.value())));
+            Boolean.valueOf(cfg.get(JobConfigProperty.DisableExternalView.name())));
       }
-      if (cfg.containsKey(JobConfigProperty.IGNORE_DEPENDENT_JOB_FAILURE.value())) {
+      if (cfg.containsKey(JobConfigProperty.IgnoreDependentJobFailure.name())) {
         b.setIgnoreDependentJobFailure(
-            Boolean.valueOf(cfg.get(JobConfigProperty.IGNORE_DEPENDENT_JOB_FAILURE.value())));
+            Boolean.valueOf(cfg.get(JobConfigProperty.IgnoreDependentJobFailure.name())));
       }
       return b;
     }
@@ -453,45 +443,45 @@ public class JobConfig {
     private void validate() {
       if (_taskConfigMap.isEmpty() && _targetResource == null) {
         throw new IllegalArgumentException(
-            String.format("%s cannot be null", JobConfigProperty.TARGET_RESOURCE));
+            String.format("%s cannot be null", JobConfigProperty.TargetResource));
       }
       if (_taskConfigMap.isEmpty() && _targetPartitionStates != null && _targetPartitionStates
           .isEmpty()) {
         throw new IllegalArgumentException(
-            String.format("%s cannot be an empty set", JobConfigProperty.TARGET_PARTITION_STATES));
+            String.format("%s cannot be an empty set", JobConfigProperty.TargetPartitionStates));
       }
       if (_taskConfigMap.isEmpty() && _command == null) {
         throw new IllegalArgumentException(
-            String.format("%s cannot be null", JobConfigProperty.COMMAND));
+            String.format("%s cannot be null", JobConfigProperty.Command));
       }
       if (_timeoutPerTask < 0) {
         throw new IllegalArgumentException(String
-            .format("%s has invalid value %s", JobConfigProperty.TIMEOUT_PER_TASK,
+            .format("%s has invalid value %s", JobConfigProperty.TimeoutPerPartition,
                 _timeoutPerTask));
       }
       if (_numConcurrentTasksPerInstance < 1) {
         throw new IllegalArgumentException(String
-            .format("%s has invalid value %s", JobConfigProperty.NUM_CONCURRENT_TASKS_PER_INSTANCE,
+            .format("%s has invalid value %s", JobConfigProperty.ConcurrentTasksPerInstance,
                 _numConcurrentTasksPerInstance));
       }
       if (_maxAttemptsPerTask < 1) {
         throw new IllegalArgumentException(String
-            .format("%s has invalid value %s", JobConfigProperty.MAX_ATTEMPTS_PER_TASK,
+            .format("%s has invalid value %s", JobConfigProperty.MaxAttemptsPerTask,
                 _maxAttemptsPerTask));
       }
       if (_maxForcedReassignmentsPerTask < 0) {
         throw new IllegalArgumentException(String
-            .format("%s has invalid value %s", JobConfigProperty.MAX_FORCED_REASSIGNMENTS_PER_TASK,
+            .format("%s has invalid value %s", JobConfigProperty.MaxForcedReassignmentsPerTask,
                 _maxForcedReassignmentsPerTask));
       }
       if (_failureThreshold < 0) {
         throw new IllegalArgumentException(String
-            .format("%s has invalid value %s", JobConfigProperty.FAILURE_THRESHOLD,
+            .format("%s has invalid value %s", JobConfigProperty.FailureThreshold,
                 _failureThreshold));
       }
       if (_workflow == null) {
         throw new IllegalArgumentException(
-            String.format("%s cannot be null", JobConfigProperty.WORKFLOW_ID));
+            String.format("%s cannot be null", JobConfigProperty.WorkflowID));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/JobDag.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobDag.java b/helix-core/src/main/java/org/apache/helix/task/JobDag.java
index 73a5e58..32e1ffa 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobDag.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobDag.java
@@ -26,7 +26,6 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
-import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -155,6 +154,10 @@ public class JobDag {
     }
   }
 
+  public int size() {
+    return _allNodes.size();
+  }
+
   /**
    * Checks that dag contains no cycles and all nodes are reachable.
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/JobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobQueue.java b/helix-core/src/main/java/org/apache/helix/task/JobQueue.java
index 0280c88..c350fee 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobQueue.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobQueue.java
@@ -29,35 +29,25 @@ import java.util.Map;
  * A named queue to which jobs can be added
  */
 public class JobQueue extends Workflow {
-  /* Config fields */
-  public static final String CAPACITY = "CAPACITY";
 
-  private final int _capacity;
-
-  private JobQueue(String name, int capacity, WorkflowConfig workflowConfig,
+  private JobQueue(String name, WorkflowConfig workflowConfig,
       Map<String, Map<String, String>> jobConfigs, Map<String, List<TaskConfig>> taskConfigs) {
     super(name, workflowConfig, jobConfigs, taskConfigs);
-    _capacity = capacity;
-    validate();
   }
 
   /**
    * Determine the number of jobs that this queue can accept before rejecting further jobs
+   * This method is deprecated, please use:
+   * JobQueue.getWorkflowConfig().getCapacity();
    * @return queue capacity
    */
+  @Deprecated
   public int getCapacity() {
-    return _capacity;
-  }
-
-  public Map<String, String> getResourceConfigMap() throws Exception {
-    Map<String, String> cfgMap = _workflowConfig.getResourceConfigMap();
-    cfgMap.put(CAPACITY, String.valueOf(_capacity));
-    return cfgMap;
+    return _workflowConfig.getCapacity();
   }
 
   /** Supports creation of a single queue */
   public static class Builder extends Workflow.Builder {
-    private int _capacity = Integer.MAX_VALUE;
     private List<String> jobs;
 
     public Builder(String name) {
@@ -65,42 +55,74 @@ public class JobQueue extends Workflow {
       jobs = new ArrayList<String>();
     }
 
-    public Builder expiry(long expiry) {
-      _expiry = expiry;
-      return this;
-    }
-
-    public Builder capacity(int capacity) {
-      _capacity = capacity;
-      return this;
-    }
-
+    /**
+     * Do not use this method, use workflowConfigBuilder.setCapacity() instead.
+     * If you set capacity via this method, the number given here
+     * will override capacity number set from other places.
+     * @param capacity
+     * @return
+     */
     @Override
-    public Builder fromMap(Map<String, String> cfg) {
-      super.fromMap(cfg);
-      if (cfg.containsKey(CAPACITY)) {
-        _capacity = Integer.parseInt(cfg.get(CAPACITY));
-      }
+    public Builder setCapacity(int capacity) {
+      super.setCapacity(capacity);
       return this;
     }
 
-    public void enqueueJob(final String job, JobConfig.Builder jobBuilder) {
-      if (jobs.size() >= _capacity) {
-        throw new HelixException("Failed to push new job to jobQueue, it is already full");
+    public Builder enqueueJob(final String job, JobConfig.Builder jobBuilder) {
+      if (_workflowConfigBuilder != null) {
+        if (jobs.size() >= _workflowConfigBuilder.getCapacity()) {
+          throw new HelixException("Failed to push new job to jobQueue, it is already full");
+        }
       }
-      addJobConfig(job, jobBuilder);
+
+      addJob(job, jobBuilder);
       if (jobs.size() > 0) {
         String previousJob = jobs.get(jobs.size() - 1);
         addParentChildDependency(previousJob, job);
       }
       jobs.add(job);
+      return this;
+    }
+
+    /**
+     * Please use setWorkflowConfigMap() instead.
+     * @param workflowCfgMap
+     * @return
+     */
+    @Override
+    public Builder fromMap(Map<String, String> workflowCfgMap) {
+      return setWorkflowConfigMap(workflowCfgMap);
+    }
+
+    @Override
+    public Builder setWorkflowConfigMap(Map<String, String> workflowCfgMap) {
+      super.setWorkflowConfigMap(workflowCfgMap);
+      return this;
+    }
+
+    @Override
+    public Builder setWorkflowConfig(WorkflowConfig workflowConfig) {
+      super.setWorkflowConfig(workflowConfig);
+      return this;
+    }
+
+    @Override
+    public Builder setScheduleConfig(ScheduleConfig scheduleConfig) {
+      super.setScheduleConfig(scheduleConfig);
+      return this;
+    }
+
+    @Override
+    public Builder setExpiry(long expiry) {
+      super.setExpiry(expiry);
+      return this;
     }
 
+    @Override
     public JobQueue build() {
-      WorkflowConfig.Builder builder = buildWorkflowConfig();
-      builder.setTerminable(false);
-      WorkflowConfig workflowConfig = builder.build();
-      return new JobQueue(_name, _capacity, workflowConfig, _jobConfigs, _taskConfigs);
+      buildConfig();
+      _workflowConfigBuilder.setTerminable(false);
+      return new JobQueue(_name, _workflowConfigBuilder.build(), _jobConfigs, _taskConfigs);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index c3eb8bd..99bcb62 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -45,7 +45,6 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.HelixProperty;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyPathConfig;
@@ -292,7 +291,8 @@ public class TaskDriver {
     DataUpdater<ZNRecord> updater = new DataUpdater<ZNRecord>() {
       @Override
       public ZNRecord update(ZNRecord currentData) {
-        JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
+        JobDag jobDag = JobDag.fromJson(
+            currentData.getSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
         for (String resourceName : toRemove) {
           for (String child : jobDag.getDirectChildren(resourceName)) {
             jobDag.getChildrenToParents().get(child).remove(resourceName);
@@ -305,7 +305,8 @@ public class TaskDriver {
           jobDag.getAllNodes().remove(resourceName);
         }
         try {
-          currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
+          currentData
+              .setSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name(), jobDag.toJson());
         } catch (Exception e) {
           throw new IllegalArgumentException(e);
         }
@@ -432,7 +433,8 @@ public class TaskDriver {
           return null;
         }
         // Add the node to the existing DAG
-        JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
+        JobDag jobDag = JobDag.fromJson(
+            currentData.getSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
         Set<String> allNodes = jobDag.getAllNodes();
         if (!allNodes.contains(namespacedJobName)) {
           LOG.warn(
@@ -458,7 +460,8 @@ public class TaskDriver {
 
         // Save the updated DAG
         try {
-          currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
+          currentData
+              .setSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name(), jobDag.toJson());
         } catch (Exception e) {
           throw new IllegalStateException(
               "Could not remove job " + jobName + " from DAG of queue " + queueName, e);
@@ -509,18 +512,16 @@ public class TaskDriver {
   public void enqueueJob(final String queueName, final String jobName,
       JobConfig.Builder jobBuilder) {
     // Get the job queue config and capacity
-    HelixProperty workflowConfig =
-        _accessor.getProperty(_accessor.keyBuilder().resourceConfig(queueName));
+    WorkflowConfig workflowConfig = TaskUtil.getWorkflowCfg(_accessor, queueName);
     if (workflowConfig == null) {
-      throw new IllegalArgumentException("Queue " + queueName + " does not yet exist!");
+      throw new IllegalArgumentException("Queue " + queueName + " config does not yet exist!");
     }
-    boolean isTerminable =
-        workflowConfig.getRecord().getBooleanField(WorkflowConfig.TERMINABLE, true);
+    boolean isTerminable = workflowConfig.isTerminable();
     if (isTerminable) {
       throw new IllegalArgumentException(queueName + " is not a queue!");
     }
-    final int capacity =
-        workflowConfig.getRecord().getIntField(JobQueue.CAPACITY, Integer.MAX_VALUE);
+
+    final int capacity = workflowConfig.getCapacity();
 
     // Create the job to ensure that it validates
     JobConfig jobConfig = jobBuilder.setWorkflow(queueName).build();
@@ -535,9 +536,10 @@ public class TaskDriver {
       @Override
       public ZNRecord update(ZNRecord currentData) {
         // Add the node to the existing DAG
-        JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
+        JobDag jobDag = JobDag.fromJson(
+            currentData.getSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
         Set<String> allNodes = jobDag.getAllNodes();
-        if (allNodes.size() >= capacity) {
+        if (capacity > 0 && allNodes.size() >= capacity) {
           throw new IllegalStateException(
               "Queue " + queueName + " is at capacity, will not add " + jobName);
         }
@@ -561,7 +563,8 @@ public class TaskDriver {
 
         // Save the updated DAG
         try {
-          currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
+          currentData
+              .setSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name(), jobDag.toJson());
         } catch (Exception e) {
           throw new IllegalStateException("Could not add job " + jobName + " to queue " + queueName,
               e);
@@ -689,7 +692,8 @@ public class TaskDriver {
           // Only update target state for non-completed workflows
           String finishTime = currentData.getSimpleField(WorkflowContext.FINISH_TIME);
           if (finishTime == null || finishTime.equals(WorkflowContext.UNFINISHED)) {
-            currentData.setSimpleField(WorkflowConfig.TARGET_STATE, state.name());
+            currentData.setSimpleField(WorkflowConfig.WorkflowConfigProperty.TargetState.name(),
+                state.name());
           } else {
             LOG.info("TargetState DataUpdater: ignore to update target state " + finishTime);
           }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
index 49622f3..513c14e 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
@@ -20,14 +20,9 @@ package org.apache.helix.task;
  */
 
 import java.io.IOException;
-import java.text.ParseException;
 import java.util.Collections;
-import java.util.Date;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.helix.AccessOption;
 import org.apache.helix.HelixDataAccessor;
@@ -36,7 +31,6 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
@@ -46,7 +40,6 @@ import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 
 import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 /**
@@ -319,34 +312,6 @@ public class TaskUtil {
     }
   }
 
-  /**
-   * Get a ScheduleConfig from a workflow config string map
-   *
-   * @param cfg the string map
-   * @return a ScheduleConfig if one exists, otherwise null
-   */
-  public static ScheduleConfig parseScheduleFromConfigMap(Map<String, String> cfg) {
-    // Parse schedule-specific configs, if they exist
-    Date startTime = null;
-    if (cfg.containsKey(WorkflowConfig.START_TIME)) {
-      try {
-        startTime = WorkflowConfig.getDefaultDateFormat().parse(cfg.get(WorkflowConfig.START_TIME));
-      } catch (ParseException e) {
-        LOG.error("Unparseable date " + cfg.get(WorkflowConfig.START_TIME), e);
-        return null;
-      }
-    }
-    if (cfg.containsKey(WorkflowConfig.RECURRENCE_UNIT) && cfg
-        .containsKey(WorkflowConfig.RECURRENCE_INTERVAL)) {
-      return ScheduleConfig
-          .recurringFromDate(startTime, TimeUnit.valueOf(cfg.get(WorkflowConfig.RECURRENCE_UNIT)),
-              Long.parseLong(cfg.get(WorkflowConfig.RECURRENCE_INTERVAL)));
-    } else if (startTime != null) {
-      return ScheduleConfig.oneTimeDelayedStart(startTime);
-    }
-    return null;
-  }
-
   private static HelixProperty getResourceConfig(HelixDataAccessor accessor, String resource) {
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
     return accessor.getProperty(keyBuilder.resourceConfig(resource));

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/Workflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/Workflow.java b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
index 1706bec..f3abc2e 100644
--- a/helix-core/src/main/java/org/apache/helix/task/Workflow.java
+++ b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
@@ -32,6 +32,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 
+import org.apache.helix.HelixException;
 import org.apache.helix.task.beans.JobBean;
 import org.apache.helix.task.beans.TaskBean;
 import org.apache.helix.task.beans.WorkflowBean;
@@ -87,6 +88,14 @@ public class Workflow {
   }
 
   /**
+   * @return Resource configuration key/value map.
+   * @throws HelixException
+   */
+  public Map<String, String> getResourceConfigMap() throws HelixException {
+    return _workflowConfig.getResourceConfigMap();
+  }
+
+  /**
    * Parses the YAML description from a file into a {@link Workflow} object.
    * @param file An abstract path name to the file containing the workflow description.
    * @return A {@link Workflow} object.
@@ -134,7 +143,7 @@ public class Workflow {
   private static Workflow parse(Reader reader) throws Exception {
     Yaml yaml = new Yaml(new Constructor(WorkflowBean.class));
     WorkflowBean wf = (WorkflowBean) yaml.load(reader);
-    Builder builder = new Builder(wf.name);
+    Builder workflowBuilder = new Builder(wf.name);
 
     if (wf != null && wf.jobs != null) {
       for (JobBean job : wf.jobs) {
@@ -144,53 +153,55 @@ public class Workflow {
 
         if (job.parents != null) {
           for (String parent : job.parents) {
-            builder.addParentChildDependency(parent, job.name);
+            workflowBuilder.addParentChildDependency(parent, job.name);
           }
         }
 
-        builder.addConfig(job.name, JobConfig.JobConfigProperty.WORKFLOW_ID.value(), wf.name);
-        builder.addConfig(job.name, JobConfig.JobConfigProperty.COMMAND.value(), job.command);
+        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.WorkflowID.name(), wf.name);
+        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.Command.name(), job.command);
         if (job.jobConfigMap != null) {
-          builder.addJobCommandConfigMap(job.name, job.jobConfigMap);
+          workflowBuilder.addJobCommandConfigMap(job.name, job.jobConfigMap);
         }
-        builder.addConfig(job.name, JobConfig.JobConfigProperty.TARGET_RESOURCE.value(),
+        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TargetResource.name(),
             job.targetResource);
         if (job.targetPartitionStates != null) {
-          builder.addConfig(job.name, JobConfig.JobConfigProperty.TARGET_PARTITION_STATES.value(),
+          workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TargetPartitionStates.name(),
               Joiner.on(",").join(job.targetPartitionStates));
         }
         if (job.targetPartitions != null) {
-          builder.addConfig(job.name, JobConfig.JobConfigProperty.TARGET_PARTITIONS.value(),
+          workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TargetPartitions.name(),
               Joiner.on(",").join(job.targetPartitions));
         }
-        builder.addConfig(job.name, JobConfig.JobConfigProperty.MAX_ATTEMPTS_PER_TASK.value(),
+        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.MaxAttemptsPerTask.name(),
             String.valueOf(job.maxAttemptsPerTask));
-        builder.addConfig(job.name,
-            JobConfig.JobConfigProperty.MAX_FORCED_REASSIGNMENTS_PER_TASK.value(),
+        workflowBuilder.addConfig(job.name,
+            JobConfig.JobConfigProperty.MaxForcedReassignmentsPerTask.name(),
             String.valueOf(job.maxForcedReassignmentsPerTask));
-        builder.addConfig(job.name,
-            JobConfig.JobConfigProperty.NUM_CONCURRENT_TASKS_PER_INSTANCE.value(),
+        workflowBuilder.addConfig(job.name,
+            JobConfig.JobConfigProperty.ConcurrentTasksPerInstance.name(),
             String.valueOf(job.numConcurrentTasksPerInstance));
-        builder.addConfig(job.name, JobConfig.JobConfigProperty.TIMEOUT_PER_TASK.value(),
+        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TimeoutPerPartition.name(),
             String.valueOf(job.timeoutPerPartition));
-        builder.addConfig(job.name, JobConfig.JobConfigProperty.FAILURE_THRESHOLD.value(),
+        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.FailureThreshold.name(),
             String.valueOf(job.failureThreshold));
         if (job.tasks != null) {
           List<TaskConfig> taskConfigs = Lists.newArrayList();
           for (TaskBean task : job.tasks) {
             taskConfigs.add(TaskConfig.from(task));
           }
-          builder.addTaskConfigs(job.name, taskConfigs);
+          workflowBuilder.addTaskConfigs(job.name, taskConfigs);
         }
       }
     }
 
+    WorkflowConfig.Builder workflowCfgBuilder = new WorkflowConfig.Builder();
     if (wf.schedule != null) {
-      builder.setScheduleConfig(ScheduleConfig.from(wf.schedule));
+      workflowCfgBuilder.setScheduleConfig(ScheduleConfig.from(wf.schedule));
     }
-    builder.setExpiry(wf.expiry);
+    workflowCfgBuilder.setExpiry(wf.expiry);
+    workflowBuilder.setWorkflowConfig(workflowCfgBuilder.build());
 
-    return builder.build();
+    return workflowBuilder.build();
   }
 
   /**
@@ -212,6 +223,13 @@ public class Workflow {
           ", names in dag but not in config: " + jobNamesInDagButNotInConfig);
     }
 
+    int capacity = _workflowConfig.getCapacity();
+    int dagSize = _workflowConfig.getJobDag().size();
+    if (capacity > 0 && dagSize > capacity) {
+      throw new IllegalArgumentException(String.format(
+          "Failed to build workflow %s, number of jobs are more than its capacity! capacity(%d), jobs(%d)",
+          _name, capacity, dagSize));
+    }
     _workflowConfig.getJobDag().validate();
 
     for (String node : _jobConfigs.keySet()) {
@@ -234,17 +252,13 @@ public class Workflow {
     protected JobDag _dag;
     protected Map<String, Map<String, String>> _jobConfigs;
     protected Map<String, List<TaskConfig>> _taskConfigs;
-    protected ScheduleConfig _scheduleConfig;
-    protected long _expiry = -1;
-    protected Map<String, String> _cfgMap;
-    protected int _parallelJobs = -1;
+    protected WorkflowConfig.Builder _workflowConfigBuilder;
 
     public Builder(String name) {
       _name = name;
       _dag = new JobDag();
       _jobConfigs = new TreeMap<String, Map<String, String>>();
       _taskConfigs = new TreeMap<String, List<TaskConfig>>();
-      _expiry = -1;
     }
 
     protected Builder addConfig(String job, String key, String val) {
@@ -258,11 +272,22 @@ public class Workflow {
     }
 
     private Builder addJobCommandConfigMap(String job, Map<String, String> jobConfigMap) {
-      return addConfig(job, JobConfig.JobConfigProperty.JOB_COMMAND_CONFIG_MAP.value(),
+      return addConfig(job, JobConfig.JobConfigProperty.JobCommandConfig.name(),
           TaskUtil.serializeJobCommandConfigMap(jobConfigMap));
     }
 
+    /**
+     * Please use addJob() instead.
+     * @param job
+     * @param jobConfigBuilder
+     * @return
+     */
+    @Deprecated
     public Builder addJobConfig(String job, JobConfig.Builder jobConfigBuilder) {
+      return addJob(job, jobConfigBuilder);
+    }
+
+    public Builder addJob(String job, JobConfig.Builder jobConfigBuilder) {
       JobConfig jobConfig = jobConfigBuilder.setWorkflow(_name).build();
       for (Map.Entry<String, String> e : jobConfig.getResourceConfigMap().entrySet()) {
         String key = e.getKey();
@@ -294,62 +319,73 @@ public class Workflow {
       return this;
     }
 
-    public Builder fromMap(Map<String, String> cfg) {
-      _cfgMap = cfg;
+    /**
+     * Please use setWorkflowConfigMap() instead.
+     * @param workflowCfgMap
+     * @return
+     */
+    public Builder fromMap(Map<String, String> workflowCfgMap) {
+      return setWorkflowConfigMap(workflowCfgMap);
+    }
+
+    public Builder setWorkflowConfigMap(Map<String, String> workflowCfgMap) {
+      if (workflowCfgMap != null && !workflowCfgMap.isEmpty()) {
+        if (_workflowConfigBuilder == null) {
+          _workflowConfigBuilder = WorkflowConfig.Builder.fromMap(workflowCfgMap);
+        } else {
+          _workflowConfigBuilder.setConfigMap(workflowCfgMap);
+        }
+      }
+      return this;
+    }
+
+    public Builder setWorkflowConfig(WorkflowConfig workflowConfig) {
+      _workflowConfigBuilder = new WorkflowConfig.Builder(workflowConfig);
       return this;
     }
 
     public Builder setScheduleConfig(ScheduleConfig scheduleConfig) {
-      _scheduleConfig = scheduleConfig;
+      if (_workflowConfigBuilder == null) {
+        _workflowConfigBuilder = new WorkflowConfig.Builder();
+      }
+      _workflowConfigBuilder.setScheduleConfig(scheduleConfig);
       return this;
     }
 
     public Builder setExpiry(long expiry) {
-      _expiry = expiry;
+      if (_workflowConfigBuilder == null) {
+        _workflowConfigBuilder = new WorkflowConfig.Builder();
+      }
+      _workflowConfigBuilder.setExpiry(expiry);
       return this;
     }
 
-    public String namespacify(String job) {
-      return TaskUtil.getNamespacedJobName(_name, job);
+    @Deprecated
+    public Builder setCapacity(int capacity) {
+      if (_workflowConfigBuilder == null) {
+        _workflowConfigBuilder = new WorkflowConfig.Builder();
+      }
+      _workflowConfigBuilder.setCapacity(capacity);
+      return this;
     }
 
-    public Builder parallelJobs(int parallelJobs) {
-      _parallelJobs = parallelJobs;
-      return this;
+    public String namespacify(String job) {
+      return TaskUtil.getNamespacedJobName(_name, job);
     }
 
     public Workflow build() {
-      WorkflowConfig.Builder builder = buildWorkflowConfig();
-      // calls validate internally
-      return new Workflow(_name, builder.build(), _jobConfigs, _taskConfigs);
+      buildConfig();
+      return new Workflow(_name, _workflowConfigBuilder.build(), _jobConfigs, _taskConfigs);
     }
 
-    protected WorkflowConfig.Builder buildWorkflowConfig() {
+    protected void buildConfig() {
       for (String task : _jobConfigs.keySet()) {
-        // addConfig(task, TaskConfig.WORKFLOW_ID, _name);
-        _jobConfigs.get(task).put(JobConfig.JobConfigProperty.WORKFLOW_ID.value(), _name);
-      }
-
-      WorkflowConfig.Builder builder;
-      if (_cfgMap != null) {
-        builder = WorkflowConfig.Builder.fromMap(_cfgMap);
-      } else {
-        builder = new WorkflowConfig.Builder();
+        _jobConfigs.get(task).put(JobConfig.JobConfigProperty.WorkflowID.name(), _name);
       }
-
-      builder.setJobDag(_dag);
-      builder.setTargetState(TargetState.START);
-      if (_scheduleConfig != null) {
-        builder.setScheduleConfig(_scheduleConfig);
+      if (_workflowConfigBuilder == null) {
+        _workflowConfigBuilder = new WorkflowConfig.Builder();
       }
-      if (_expiry > 0) {
-        builder.setExpiry(_expiry);
-      }
-      if (_parallelJobs > 0) {
-        builder.setParallelJobs(_parallelJobs);
-      }
-
-      return builder;
+      _workflowConfigBuilder.setJobDag(_dag);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index 955cb77..db9fdba 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -19,6 +19,7 @@ package org.apache.helix.task;
  * under the License.
  */
 import java.io.IOException;
+import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
@@ -26,21 +27,34 @@ import java.util.Map;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.HelixException;
+import org.apache.log4j.Logger;
 
 /**
  * Provides a typed interface to workflow level configurations. Validates the configurations.
  */
 public class WorkflowConfig {
-  /* Config fields */
-  public static final String DAG = "Dag";
-  public static final String PARALLEL_JOBS = "ParallelJobs";
-  public static final String TARGET_STATE = "TargetState";
-  public static final String EXPIRY = "Expiry";
-  public static final String START_TIME = "StartTime";
-  public static final String RECURRENCE_UNIT = "RecurrenceUnit";
-  public static final String RECURRENCE_INTERVAL = "RecurrenceInterval";
-  public static final String TERMINABLE = "Terminable";
-  public static final String FAILURE_THRESHOLD = "FailureThreshold";
+  private static final Logger LOG = Logger.getLogger(WorkflowConfig.class);
+
+  /**
+   * Do not use these values directly, always use the getters/setters
+   * in WorkflowConfig and WorkflowConfig.Builder.
+   *
+   * For back-compatible, this class will be left for public for a while,
+   * but it will be change to protected in future major release.
+   */
+  public enum WorkflowConfigProperty {
+    Dag,
+    ParallelJobs,
+    TargetState,
+    Expiry,
+    StartTime,
+    RecurrenceUnit,
+    RecurrenceInterval,
+    Terminable,
+    FailureThreshold,
+    /* this is for non-terminable workflow. */
+    capacity
+  }
 
   /* Default values */
   public static final long DEFAULT_EXPIRY = 24 * 60 * 60 * 1000;
@@ -59,9 +73,10 @@ public class WorkflowConfig {
   private final boolean _terminable;
   private final ScheduleConfig _scheduleConfig;
   private final int _failureThreshold;
+  private final int _capacity;
 
   protected WorkflowConfig(JobDag jobDag, int parallelJobs, TargetState targetState, long expiry,
-      int failureThreshold, boolean terminable, ScheduleConfig scheduleConfig) {
+      int failureThreshold, boolean terminable, ScheduleConfig scheduleConfig, int capacity) {
     _jobDag = jobDag;
     _parallelJobs = parallelJobs;
     _targetState = targetState;
@@ -69,6 +84,7 @@ public class WorkflowConfig {
     _failureThreshold = failureThreshold;
     _terminable = terminable;
     _scheduleConfig = scheduleConfig;
+    _capacity = capacity;
   }
 
   public JobDag getJobDag() {
@@ -91,6 +107,13 @@ public class WorkflowConfig {
     return _failureThreshold;
   }
 
+  /**
+   * Determine the number of jobs that this workflow can accept before rejecting further jobs,
+   * this field is only used when a workflow is not terminable.
+   * @return queue capacity
+   */
+  public int getCapacity() { return _capacity; }
+
   public boolean isTerminable() {
     return _terminable;
   }
@@ -128,15 +151,20 @@ public class WorkflowConfig {
   public Map<String, String> getResourceConfigMap() {
     Map<String, String> cfgMap = new HashMap<String, String>();
     try {
-      cfgMap.put(WorkflowConfig.DAG, getJobDag().toJson());
+      cfgMap.put(WorkflowConfigProperty.Dag.name(), getJobDag().toJson());
     } catch (IOException ex) {
       throw new HelixException("Invalid job dag configuration!", ex);
     }
-    cfgMap.put(WorkflowConfig.PARALLEL_JOBS, String.valueOf(getParallelJobs()));
-    cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(getExpiry()));
-    cfgMap.put(WorkflowConfig.TARGET_STATE, getTargetState().name());
-    cfgMap.put(WorkflowConfig.TERMINABLE, String.valueOf(isTerminable()));
-    cfgMap.put(WorkflowConfig.FAILURE_THRESHOLD, String.valueOf(getFailureThreshold()));
+    cfgMap.put(WorkflowConfigProperty.ParallelJobs.name(), String.valueOf(getParallelJobs()));
+    cfgMap.put(WorkflowConfigProperty.Expiry.name(), String.valueOf(getExpiry()));
+    cfgMap.put(WorkflowConfigProperty.TargetState.name(), getTargetState().name());
+    cfgMap.put(WorkflowConfigProperty.Terminable.name(), String.valueOf(isTerminable()));
+    cfgMap.put(WorkflowConfigProperty.FailureThreshold.name(),
+        String.valueOf(getFailureThreshold()));
+
+    if (_capacity > 0) {
+      cfgMap.put(WorkflowConfigProperty.capacity.name(), String.valueOf(_capacity));
+    }
 
     // Populate schedule if present
     ScheduleConfig scheduleConfig = getScheduleConfig();
@@ -144,17 +172,50 @@ public class WorkflowConfig {
       Date startTime = scheduleConfig.getStartTime();
       if (startTime != null) {
         String formattedTime = WorkflowConfig.getDefaultDateFormat().format(startTime);
-        cfgMap.put(WorkflowConfig.START_TIME, formattedTime);
+        cfgMap.put(WorkflowConfigProperty.StartTime.name(), formattedTime);
       }
       if (scheduleConfig.isRecurring()) {
-        cfgMap.put(WorkflowConfig.RECURRENCE_UNIT, scheduleConfig.getRecurrenceUnit().toString());
-        cfgMap.put(WorkflowConfig.RECURRENCE_INTERVAL, scheduleConfig.getRecurrenceInterval()
-            .toString());
+        cfgMap.put(WorkflowConfigProperty.RecurrenceUnit.name(),
+            scheduleConfig.getRecurrenceUnit().toString());
+        cfgMap.put(WorkflowConfigProperty.RecurrenceInterval.name(),
+            scheduleConfig.getRecurrenceInterval().toString());
       }
     }
     return cfgMap;
   }
 
+  /**
+   * Get a ScheduleConfig from a workflow config string map
+   *
+   * @param cfg the string map
+   * @return a ScheduleConfig if one exists, otherwise null
+   */
+  public static ScheduleConfig parseScheduleFromConfigMap(Map<String, String> cfg) {
+    // Parse schedule-specific configs, if they exist
+    Date startTime = null;
+    if (cfg.containsKey(WorkflowConfigProperty.StartTime.name())) {
+      try {
+        startTime = WorkflowConfig.getDefaultDateFormat()
+            .parse(cfg.get(WorkflowConfigProperty.StartTime.name()));
+      } catch (ParseException e) {
+        LOG.error(
+            "Unparseable date " + cfg.get(WorkflowConfigProperty.StartTime.name()),
+            e);
+        return null;
+      }
+    }
+    if (cfg.containsKey(WorkflowConfigProperty.RecurrenceUnit.name()) && cfg
+        .containsKey(WorkflowConfigProperty.RecurrenceInterval.name())) {
+      return ScheduleConfig.recurringFromDate(startTime,
+          TimeUnit.valueOf(cfg.get(WorkflowConfigProperty.RecurrenceUnit.name())),
+          Long.parseLong(
+              cfg.get(WorkflowConfigProperty.RecurrenceInterval.name())));
+    } else if (startTime != null) {
+      return ScheduleConfig.oneTimeDelayedStart(startTime);
+    }
+    return null;
+  }
+
   public static class Builder {
     private JobDag _taskDag = JobDag.EMPTY_DAG;
     private int _parallelJobs = 1;
@@ -162,13 +223,14 @@ public class WorkflowConfig {
     private long _expiry = DEFAULT_EXPIRY;
     private int _failureThreshold = DEFAULT_FAILURE_THRESHOLD;
     private boolean _isTerminable = true;
+    private int _capacity = Integer.MAX_VALUE;
     private ScheduleConfig _scheduleConfig;
 
     public WorkflowConfig build() {
       validate();
 
       return new WorkflowConfig(_taskDag, _parallelJobs, _targetState, _expiry, _failureThreshold,
-          _isTerminable, _scheduleConfig);
+          _isTerminable, _scheduleConfig, _capacity);
     }
 
     public Builder() {}
@@ -180,9 +242,11 @@ public class WorkflowConfig {
       _expiry = workflowConfig.getExpiry();
       _isTerminable = workflowConfig.isTerminable();
       _scheduleConfig = workflowConfig.getScheduleConfig();
+      _capacity = workflowConfig.getCapacity();
+      _failureThreshold = workflowConfig.getFailureThreshold();
     }
 
-    public Builder setJobDag(JobDag v) {
+    protected Builder setJobDag(JobDag v) {
       _taskDag = v;
       return this;
     }
@@ -207,6 +271,11 @@ public class WorkflowConfig {
       return this;
     }
 
+    public Builder setCapacity(int capacity) {
+      _capacity = capacity;
+      return this;
+    }
+
     public Builder setTerminable(boolean isTerminable) {
       _isTerminable = isTerminable;
       return this;
@@ -223,43 +292,95 @@ public class WorkflowConfig {
     }
 
     public static Builder fromMap(Map<String, String> cfg) {
-      Builder b = new Builder();
-      if (cfg.containsKey(EXPIRY)) {
-        b.setExpiry(Long.parseLong(cfg.get(EXPIRY)));
+      Builder builder = new Builder();
+      builder.setConfigMap(cfg);
+      return builder;
+    }
+
+    public Builder setConfigMap(Map<String, String> cfg) {
+      if (cfg.containsKey(WorkflowConfigProperty.Expiry.name())) {
+        setExpiry(Long.parseLong(cfg.get(WorkflowConfigProperty.Expiry.name())));
       }
-      if (cfg.containsKey(FAILURE_THRESHOLD)) {
-        b.setFailureThreshold(Integer.parseInt(cfg.get(FAILURE_THRESHOLD)));
+      if (cfg.containsKey(WorkflowConfigProperty.FailureThreshold.name())) {
+        setFailureThreshold(
+            Integer.parseInt(cfg.get(WorkflowConfigProperty.FailureThreshold.name())));
       }
-      if (cfg.containsKey(DAG)) {
-        b.setJobDag(JobDag.fromJson(cfg.get(DAG)));
+      if (cfg.containsKey(WorkflowConfigProperty.Dag.name())) {
+        setJobDag(JobDag.fromJson(cfg.get(WorkflowConfigProperty.Dag.name())));
       }
-      if (cfg.containsKey(TARGET_STATE)) {
-        b.setTargetState(TargetState.valueOf(cfg.get(TARGET_STATE)));
+      if (cfg.containsKey(WorkflowConfigProperty.TargetState.name())) {
+        setTargetState(TargetState.valueOf(cfg.get(WorkflowConfigProperty.TargetState.name())));
       }
-      if (cfg.containsKey(TERMINABLE)) {
-        b.setTerminable(Boolean.parseBoolean(cfg.get(TERMINABLE)));
+      if (cfg.containsKey(WorkflowConfigProperty.Terminable.name())) {
+        setTerminable(Boolean.parseBoolean(cfg.get(WorkflowConfigProperty.Terminable.name())));
       }
-      if (cfg.containsKey(PARALLEL_JOBS)) {
-        String value = cfg.get(PARALLEL_JOBS);
+      if (cfg.containsKey(WorkflowConfigProperty.ParallelJobs.name())) {
+        String value = cfg.get(WorkflowConfigProperty.ParallelJobs.name());
         if (value == null) {
-          b.setParallelJobs(1);
+          setParallelJobs(1);
         } else {
-          b.setParallelJobs(Integer.parseInt(value));
+          setParallelJobs(Integer.parseInt(value));
+        }
+      }
+
+      if (cfg.containsKey(WorkflowConfigProperty.capacity.name())) {
+        int capacity = Integer.valueOf(cfg.get(WorkflowConfigProperty.capacity.name()));
+        if (capacity > 0) {
+          setCapacity(capacity);
+        }
+      }
+
+      if (cfg.containsKey(WorkflowConfigProperty.FailureThreshold.name())) {
+        int threshold = Integer.valueOf(cfg.get(WorkflowConfigProperty.FailureThreshold.name()));
+        if (threshold >= 0) {
+          setFailureThreshold(threshold);
         }
       }
 
       // Parse schedule-specific configs, if they exist
-      ScheduleConfig scheduleConfig = TaskUtil.parseScheduleFromConfigMap(cfg);
+      ScheduleConfig scheduleConfig = parseScheduleFromConfigMap(cfg);
       if (scheduleConfig != null) {
-        b.setScheduleConfig(scheduleConfig);
+        setScheduleConfig(scheduleConfig);
       }
-      return b;
+      return this;
+    }
+
+    public int getParallelJobs() {
+      return _parallelJobs;
+    }
+
+    public TargetState getTargetState() {
+      return _targetState;
+    }
+
+    public long getExpiry() {
+      return _expiry;
+    }
+
+    public int getFailureThreshold() {
+      return _failureThreshold;
+    }
+
+    public boolean isTerminable() {
+      return _isTerminable;
+    }
+
+    public int getCapacity() {
+      return _capacity;
+    }
+
+    public ScheduleConfig getScheduleConfig() {
+      return _scheduleConfig;
+    }
+
+    public JobDag getJobDag() {
+      return _taskDag;
     }
 
     private void validate() {
       if (_expiry < 0) {
-        throw new IllegalArgumentException(
-            String.format("%s has invalid value %s", EXPIRY, _expiry));
+        throw new IllegalArgumentException(String
+            .format("%s has invalid value %s", WorkflowConfigProperty.Expiry.name(), _expiry));
       } else if (_scheduleConfig != null && !_scheduleConfig.isValid()) {
         throw new IllegalArgumentException(
             "Scheduler configuration is invalid. The configuration must have a start time if it is "

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
index 682ac77..9d1106a 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
@@ -75,11 +75,6 @@ public class WorkflowRebalancer extends TaskRebalancer {
 
     if (targetState == TargetState.STOP) {
       LOG.info("Workflow " + workflow + "is marked as stopped.");
-      // Workflow has been stopped if all jobs are stopped
-      // TODO: what should we do if workflowCtx is not set yet?
-      if (workflowCtx != null && isWorkflowStopped(workflowCtx, workflowCfg)) {
-        workflowCtx.setWorkflowState(TaskState.STOPPED);
-      }
       return buildEmptyAssignment(workflow, currStateOutput);
     }
 
@@ -339,25 +334,24 @@ public class WorkflowRebalancer extends TaskRebalancer {
     }
 
     // Create a new workflow with a new name
-    HelixProperty workflowConfig = resourceConfigMap.get(origWorkflowName);
-    Map<String, String> wfSimpleFields = workflowConfig.getRecord().getSimpleFields();
-    JobDag jobDag = JobDag.fromJson(wfSimpleFields.get(WorkflowConfig.DAG));
-    Map<String, Set<String>> parentsToChildren = jobDag.getParentsToChildren();
-    Workflow.Builder workflowBuilder = new Workflow.Builder(newWorkflowName);
-
-    // Set the workflow expiry
-    workflowBuilder.setExpiry(Long.parseLong(wfSimpleFields.get(WorkflowConfig.EXPIRY)));
+    Map<String, String> workflowConfigsMap =
+        resourceConfigMap.get(origWorkflowName).getRecord().getSimpleFields();
+    WorkflowConfig.Builder workflowConfigBlder = WorkflowConfig.Builder.fromMap(workflowConfigsMap);
 
     // Set the schedule, if applicable
-    ScheduleConfig scheduleConfig;
     if (newStartTime != null) {
-      scheduleConfig = ScheduleConfig.oneTimeDelayedStart(newStartTime);
-    } else {
-      scheduleConfig = TaskUtil.parseScheduleFromConfigMap(wfSimpleFields);
-    }
-    if (scheduleConfig != null) {
-      workflowBuilder.setScheduleConfig(scheduleConfig);
+      ScheduleConfig scheduleConfig = ScheduleConfig.oneTimeDelayedStart(newStartTime);
+      workflowConfigBlder.setScheduleConfig(scheduleConfig);
     }
+    workflowConfigBlder.setTerminable(true);
+
+    WorkflowConfig workflowConfig = workflowConfigBlder.build();
+
+    JobDag jobDag = workflowConfig.getJobDag();
+    Map<String, Set<String>> parentsToChildren = jobDag.getParentsToChildren();
+
+    Workflow.Builder workflowBuilder = new Workflow.Builder(newWorkflowName);
+    workflowBuilder.setWorkflowConfig(workflowConfig);
 
     // Add each job back as long as the original exists
     Set<String> namespacedJobs = jobDag.getAllNodes();
@@ -454,10 +448,10 @@ public class WorkflowRebalancer extends TaskRebalancer {
     // Remove DAG references in workflow
     PropertyKey workflowKey = TaskUtil.getWorkflowConfigKey(accessor, workflow);
     DataUpdater<ZNRecord> dagRemover = new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord currentData) {
+      @Override public ZNRecord update(ZNRecord currentData) {
         if (currentData != null) {
-          JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
+          JobDag jobDag = JobDag.fromJson(
+              currentData.getSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
           for (String child : jobDag.getDirectChildren(job)) {
             jobDag.getChildrenToParents().get(child).remove(job);
           }
@@ -468,7 +462,8 @@ public class WorkflowRebalancer extends TaskRebalancer {
           jobDag.getParentsToChildren().remove(job);
           jobDag.getAllNodes().remove(job);
           try {
-            currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
+            currentData
+                .setSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name(), jobDag.toJson());
           } catch (Exception e) {
             LOG.error("Could not update DAG for job: " + job, e);
           }

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index 422ec88..3e5385c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -22,20 +22,19 @@ import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.helix.HelixManager;
 import org.apache.helix.TestHelper;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.ScheduleConfig;
+import org.apache.helix.task.TargetState;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskPartitionState;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
 import org.testng.Assert;
@@ -121,8 +120,7 @@ public class TaskTestUtil {
     final String namespacedJobName = String.format("%s_%s", workflowName, jobName);
     boolean succeed = TestHelper.verify(new TestHelper.Verifier() {
 
-      @Override
-      public boolean verify() throws Exception {
+      @Override public boolean verify() throws Exception {
         WorkflowContext ctx = driver.getWorkflowContext(workflowName);
         return ctx == null || ctx.getJobState(namespacedJobName) == null;
       }
@@ -242,36 +240,46 @@ public class TaskTestUtil {
 
   public static JobQueue.Builder buildRecurrentJobQueue(String jobQueueName, int delayStart,
       int recurrenInSeconds) {
-    Map<String, String> cfgMap = new HashMap<String, String>();
-    cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(120000));
-    cfgMap.put(WorkflowConfig.RECURRENCE_INTERVAL, String.valueOf(recurrenInSeconds));
-    cfgMap.put(WorkflowConfig.RECURRENCE_UNIT, "SECONDS");
+    return buildRecurrentJobQueue(jobQueueName, delayStart, recurrenInSeconds, null);
+  }
+
+  public static JobQueue.Builder buildRecurrentJobQueue(String jobQueueName, int delayStart,
+      int recurrenInSeconds, TargetState targetState) {
+    WorkflowConfig.Builder workflowCfgBuilder = new WorkflowConfig.Builder();
+    workflowCfgBuilder.setExpiry(120000);
+    if (targetState != null) {
+      workflowCfgBuilder.setTargetState(TargetState.STOP);
+    }
+
     Calendar cal = Calendar.getInstance();
     cal.set(Calendar.MINUTE, cal.get(Calendar.MINUTE) + delayStart / 60);
     cal.set(Calendar.SECOND, cal.get(Calendar.SECOND) + delayStart % 60);
     cal.set(Calendar.MILLISECOND, 0);
-    cfgMap.put(WorkflowConfig.START_TIME,
-        WorkflowConfig.getDefaultDateFormat().format(cal.getTime()));
-    return new JobQueue.Builder(jobQueueName).fromMap(cfgMap);
+    ScheduleConfig scheduleConfig =
+        ScheduleConfig.recurringFromDate(cal.getTime(), TimeUnit.SECONDS, recurrenInSeconds);
+    workflowCfgBuilder.setScheduleConfig(scheduleConfig);
+    return new JobQueue.Builder(jobQueueName).setWorkflowConfig(workflowCfgBuilder.build());
   }
 
   public static JobQueue.Builder buildRecurrentJobQueue(String jobQueueName) {
     return buildRecurrentJobQueue(jobQueueName, 0);
   }
 
-  public static JobQueue.Builder buildJobQueue(String jobQueueName, int delayStart, int failureThreshold) {
-    Map<String, String> cfgMap = new HashMap<String, String>();
-    cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(120000));
+  public static JobQueue.Builder buildJobQueue(String jobQueueName, int delayStart,
+      int failureThreshold) {
+    WorkflowConfig.Builder workflowCfgBuilder = new WorkflowConfig.Builder();
+    workflowCfgBuilder.setExpiry(120000);
+
     Calendar cal = Calendar.getInstance();
     cal.set(Calendar.MINUTE, cal.get(Calendar.MINUTE) + delayStart / 60);
     cal.set(Calendar.SECOND, cal.get(Calendar.SECOND) + delayStart % 60);
     cal.set(Calendar.MILLISECOND, 0);
-    cfgMap.put(WorkflowConfig.START_TIME,
-        WorkflowConfig.getDefaultDateFormat().format(cal.getTime()));
+    workflowCfgBuilder.setScheduleConfig(ScheduleConfig.oneTimeDelayedStart(cal.getTime()));
+
     if (failureThreshold > 0) {
-      cfgMap.put(WorkflowConfig.FAILURE_THRESHOLD, String.valueOf(failureThreshold));
+      workflowCfgBuilder.setFailureThreshold(failureThreshold);
     }
-    return new JobQueue.Builder(jobQueueName).fromMap(cfgMap);
+    return new JobQueue.Builder(jobQueueName).setWorkflowConfig(workflowCfgBuilder.build());
   }
 
   public static JobQueue.Builder buildJobQueue(String jobQueueName) {

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index ae3d52d..b2e61ca 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -40,14 +40,13 @@ import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.TargetState;
 import org.apache.helix.task.Task;
 import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.TaskUtil;
-import org.apache.helix.task.Workflow;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
 import org.apache.helix.tools.ClusterSetup;
@@ -237,11 +236,9 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
     for (int i = 0; i <= 4; i++) {
       String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
 
-      JobConfig.Builder job =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
-              .setJobCommandConfigMap(commandConfig)
-              .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-              .setTargetPartitionStates(Sets.newHashSet(targetPartition));
+      JobConfig.Builder job = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+          .setJobCommandConfigMap(commandConfig).setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+          .setTargetPartitionStates(Sets.newHashSet(targetPartition));
       String jobName = targetPartition.toLowerCase() + "Job" + i;
       LOG.info("Enqueuing job: " + jobName);
       queueBuilder.enqueueJob(jobName, job);
@@ -331,10 +328,9 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
     for (int i = 0; i < JOB_COUNTS; i++) {
       String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
 
-      JobConfig.Builder job =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setJobCommandConfigMap(commandConfig)
-              .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-              .setTargetPartitionStates(Sets.newHashSet(targetPartition));
+      JobConfig.Builder job = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+          .setJobCommandConfigMap(commandConfig).setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+          .setTargetPartitionStates(Sets.newHashSet(targetPartition));
       jobs.add(job);
       jobNames.add(targetPartition.toLowerCase() + "Job" + i);
     }
@@ -370,6 +366,42 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
   }
 
   @Test
+  public void testCreateStoppedQueue() throws InterruptedException {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuild = TaskTestUtil.buildRecurrentJobQueue(queueName, 0, 600000,
+        TargetState.STOP);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i <= 1; i++) {
+      String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
+
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+              .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+              .setTargetPartitionStates(Sets.newHashSet(targetPartition));
+      String jobName = targetPartition.toLowerCase() + "Job" + i;
+      queueBuild.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.createQueue(queueBuild.build());
+    WorkflowConfig workflowConfig = _driver.getWorkflowConfig(queueName);
+    Assert.assertEquals(workflowConfig.getTargetState(), TargetState.STOP);
+
+    _driver.resume(queueName);
+
+    //TaskTestUtil.pollForWorkflowState(_driver, queueName, );
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
+
+    // ensure current schedule is started
+    String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.COMPLETED);
+  }
+
+  @Test
   public void testGetNoExistWorkflowConfig() {
     String randomName = "randomJob";
     WorkflowConfig workflowConfig = _driver.getWorkflowConfig(randomName);

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
index 580f5ac..b091748 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
@@ -41,6 +41,7 @@ import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -136,14 +137,16 @@ public class TestTaskRebalancerParallel extends ZkIntegrationTestBase {
     }
   }
 
-  @Test
-  public void test() throws Exception {
+  @Test public void test() throws Exception {
     final int PARALLEL_COUNT = 2;
 
     String queueName = TestHelper.getTestMethodName();
 
-    JobQueue.Builder queueBuild = new JobQueue.Builder(queueName);
-    queueBuild.parallelJobs(PARALLEL_COUNT);
+    WorkflowConfig.Builder cfgBuilder = new WorkflowConfig.Builder();
+    cfgBuilder.setParallelJobs(PARALLEL_COUNT);
+
+    JobQueue.Builder queueBuild =
+        new JobQueue.Builder(queueName).setWorkflowConfig(cfgBuilder.build());
     JobQueue queue = queueBuild.build();
     _driver.createQueue(queue);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
index 964f9e1..2e53b36 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
@@ -34,6 +34,7 @@ import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.ScheduleConfig;
+import org.apache.helix.task.TargetState;
 import org.apache.helix.task.Task;
 import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskDriver;
@@ -50,10 +51,8 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
@@ -63,7 +62,6 @@ public class TestUpdateWorkflow extends ZkIntegrationTestBase {
   private static final int START_PORT = 12918;
   private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
   private static final String TIMEOUT_CONFIG = "Timeout";
-  private static final String TGT_DB = "TestDB";
   private static final int NUM_PARTITIONS = 20;
   private static final int NUM_REPLICAS = 3;
   private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
@@ -92,8 +90,9 @@ public class TestUpdateWorkflow extends ZkIntegrationTestBase {
     }
 
     // Set up target db
-    setupTool.addResourceToCluster(CLUSTER_NAME, TGT_DB, NUM_PARTITIONS, MASTER_SLAVE_STATE_MODEL);
-    setupTool.rebalanceStorageCluster(CLUSTER_NAME, TGT_DB, NUM_REPLICAS);
+    setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_PARTITIONS,
+        MASTER_SLAVE_STATE_MODEL);
+    setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_REPLICAS);
 
     Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
     taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
@@ -151,27 +150,13 @@ public class TestUpdateWorkflow extends ZkIntegrationTestBase {
   }
 
   @Test
-  public void testUpdateQueueConfig() throws InterruptedException {
+  public void testUpdateRunningQueue() throws InterruptedException {
     String queueName = TestHelper.getTestMethodName();
 
     // Create a queue
     LOG.info("Starting job-queue: " + queueName);
-    JobQueue.Builder queueBuild = TaskTestUtil.buildRecurrentJobQueue(queueName, 0, 600000);
-    // Create and Enqueue jobs
-    List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i <= 1; i++) {
-      String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
-
-      JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
-              .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-              .setTargetPartitionStates(Sets.newHashSet(targetPartition));
-      String jobName = targetPartition.toLowerCase() + "Job" + i;
-      queueBuild.enqueueJob(jobName, jobConfig);
-      currentJobNames.add(jobName);
-    }
-
-    _driver.start(queueBuild.build());
+    JobQueue queue = createDefaultRecurrentJobQueue(queueName, 2);
+    _driver.start(queue);
 
     WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
 
@@ -209,5 +194,75 @@ public class TestUpdateWorkflow extends ZkIntegrationTestBase {
             startTime.get(Calendar.MINUTE) == configStartTime.get(Calendar.MINUTE) &&
             startTime.get(Calendar.SECOND) == configStartTime.get(Calendar.SECOND)));
   }
+
+  @Test
+  public void testUpdateStoppedQueue() throws InterruptedException {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue queue = createDefaultRecurrentJobQueue(queueName, 2);
+    _driver.start(queue);
+
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
+
+    // ensure current schedule is started
+    String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.IN_PROGRESS);
+
+    _driver.stop(queueName);
+
+    WorkflowConfig workflowConfig = _driver.getWorkflowConfig(queueName);
+    Assert.assertEquals(workflowConfig.getTargetState(), TargetState.STOP);
+
+    WorkflowConfig.Builder configBuilder = new WorkflowConfig.Builder(workflowConfig);
+    Calendar startTime = Calendar.getInstance();
+    startTime.set(Calendar.SECOND, startTime.get(Calendar.SECOND) + 1);
+
+    ScheduleConfig scheduleConfig =
+        ScheduleConfig.recurringFromDate(startTime.getTime(), TimeUnit.MINUTES, 2);
+
+    configBuilder.setScheduleConfig(scheduleConfig);
+
+    _driver.updateWorkflow(queueName, configBuilder.build());
+
+    workflowConfig = _driver.getWorkflowConfig(queueName);
+    Assert.assertEquals(workflowConfig.getTargetState(), TargetState.STOP);
+
+    _driver.resume(queueName);
+
+    // ensure current schedule is completed
+    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.COMPLETED);
+
+    Thread.sleep(1000);
+
+    wCtx = TaskTestUtil.pollForWorkflowContext(_driver, queueName);
+    scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
+    WorkflowConfig wCfg = _driver.getWorkflowConfig(scheduledQueue);
+
+    Calendar configStartTime = Calendar.getInstance();
+    configStartTime.setTime(wCfg.getStartTime());
+
+    Assert.assertTrue(
+        (startTime.get(Calendar.HOUR_OF_DAY) == configStartTime.get(Calendar.HOUR_OF_DAY) &&
+            startTime.get(Calendar.MINUTE) == configStartTime.get(Calendar.MINUTE) &&
+            startTime.get(Calendar.SECOND) == configStartTime.get(Calendar.SECOND)));
+  }
+
+  private JobQueue createDefaultRecurrentJobQueue(String queueName, int numJobs) {
+    JobQueue.Builder queueBuild = TaskTestUtil.buildRecurrentJobQueue(queueName, 0, 600000);
+    for (int i = 0; i <= numJobs; i++) {
+      String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
+
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+              .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+              .setTargetPartitionStates(Sets.newHashSet(targetPartition));
+      String jobName = targetPartition.toLowerCase() + "Job" + i;
+      queueBuild.enqueueJob(jobName, jobConfig);
+    }
+
+    return queueBuild.build();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d386aff3/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java b/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
index ce3a36a..639cdff 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
@@ -31,8 +31,6 @@ import org.apache.log4j.Logger;
  * Convenience class for generating various test workflows
  */
 public class WorkflowGenerator {
-  private static final Logger LOG = Logger.getLogger(WorkflowGenerator.class);
-
   public static final String DEFAULT_TGT_DB = "TestDB";
   public static final String JOB_NAME_1 = "SomeJob1";
   public static final String JOB_NAME_2 = "SomeJob2";
@@ -54,13 +52,6 @@ public class WorkflowGenerator {
     DEFAULT_COMMAND_CONFIG = Collections.unmodifiableMap(tmpMap);
   }
 
-  private static final JobConfig.Builder DEFAULT_JOB_BUILDER;
-  static {
-    JobConfig.Builder builder = JobConfig.Builder.fromMap(DEFAULT_JOB_CONFIG);
-    builder.setJobCommandConfigMap(DEFAULT_COMMAND_CONFIG);
-    DEFAULT_JOB_BUILDER = builder;
-  }
-
   public static Workflow.Builder generateDefaultSingleJobWorkflowBuilder(String jobName) {
     JobConfig.Builder jobBuilder = JobConfig.Builder.fromMap(DEFAULT_JOB_CONFIG);
     jobBuilder.setJobCommandConfigMap(DEFAULT_COMMAND_CONFIG);


[19/33] helix git commit: Add WorkflowType and JobType in WorkflowConfig and JobConfig

Posted by lx...@apache.org.
Add WorkflowType and JobType in WorkflowConfig and JobConfig

Add WorkflowType and JobType in WorkflowConfig and JobConfig

RB=710119
G=nuage-reviewers
R=lxia,cji
A=lxia


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

Branch: refs/heads/helix-0.6.x
Commit: be78a4f14076736cca511b1d3116546162e026fc
Parents: 99a4008
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Apr 21 13:04:22 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 15:03:23 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobConfig.java   | 34 +++++++++--
 .../org/apache/helix/task/WorkflowConfig.java   | 27 ++++++++-
 .../org/apache/helix/task/beans/JobBean.java    |  1 +
 .../apache/helix/task/beans/WorkflowBean.java   |  1 +
 .../task/TestJobAndWorkflowType.java            | 60 ++++++++++++++++++++
 5 files changed, 116 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/be78a4f1/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index ac086d3..1eeca60 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -113,7 +113,12 @@ public class JobConfig {
     /**
      * Disable external view (not showing) for this job resource
      */
-    DisableExternalView
+    DisableExternalView,
+
+    /**
+     * The type of the job
+     */
+    JobType
   }
 
   //Default property values
@@ -128,6 +133,7 @@ public class JobConfig {
 
   private final String _workflow;
   private final String _targetResource;
+  private final String _jobType;
   private final List<String> _targetPartitions;
   private final Set<String> _targetPartitionStates;
   private final String _command;
@@ -146,7 +152,8 @@ public class JobConfig {
       Set<String> targetPartitionStates, String command, Map<String, String> jobCommandConfigMap,
       long timeoutPerTask, int numConcurrentTasksPerInstance, int maxAttemptsPerTask,
       int maxForcedReassignmentsPerTask, int failureThreshold, long retryDelay,
-      boolean disableExternalView, boolean ignoreDependentJobFailure, Map<String, TaskConfig> taskConfigMap) {
+      boolean disableExternalView, boolean ignoreDependentJobFailure,
+      Map<String, TaskConfig> taskConfigMap, String jobType) {
     _workflow = workflow;
     _targetResource = targetResource;
     _targetPartitions = targetPartitions;
@@ -166,6 +173,7 @@ public class JobConfig {
     } else {
       _taskConfigMap = Collections.emptyMap();
     }
+    _jobType = jobType;
   }
 
   public String getWorkflow() {
@@ -267,9 +275,16 @@ public class JobConfig {
         "" + _numConcurrentTasksPerInstance);
     cfgMap.put(JobConfigProperty.IgnoreDependentJobFailure.name(),
         Boolean.toString(_ignoreDependentJobFailure));
+   if (_jobType != null) {
+     cfgMap.put(JobConfigProperty.JobType.name(), _jobType);
+   }
     return cfgMap;
   }
 
+  public String getJobType() {
+    return _jobType;
+  }
+
   public static JobConfig fromHelixProperty(HelixProperty property)
       throws IllegalArgumentException {
     Map<String, String> configs = property.getRecord().getSimpleFields();
@@ -282,6 +297,7 @@ public class JobConfig {
   public static class Builder {
     private String _workflow;
     private String _targetResource;
+    private String _jobType;
     private List<String> _targetPartitions;
     private Set<String> _targetPartitionStates;
     private String _command;
@@ -302,7 +318,7 @@ public class JobConfig {
       return new JobConfig(_workflow, _targetResource, _targetPartitions, _targetPartitionStates,
           _command, _commandConfig, _timeoutPerTask, _numConcurrentTasksPerInstance,
           _maxAttemptsPerTask, _maxForcedReassignmentsPerTask, _failureThreshold, _retryDelay,
-          _disableExternalView, _ignoreDependentJobFailure, _taskConfigMap);
+          _disableExternalView, _ignoreDependentJobFailure, _taskConfigMap, _jobType);
     }
 
     /**
@@ -364,6 +380,9 @@ public class JobConfig {
         b.setIgnoreDependentJobFailure(
             Boolean.valueOf(cfg.get(JobConfigProperty.IgnoreDependentJobFailure.name())));
       }
+      if (cfg.containsKey(JobConfigProperty.JobType.name())) {
+        b.setJobType(cfg.get(JobConfigProperty.JobType.name()));
+      }
       return b;
     }
 
@@ -451,6 +470,11 @@ public class JobConfig {
       return this;
     }
 
+    public Builder setJobType(String jobType) {
+      _jobType = jobType;
+      return this;
+    }
+
     private void validate() {
       if (_taskConfigMap.isEmpty() && _targetResource == null) {
         throw new IllegalArgumentException(
@@ -529,7 +553,9 @@ public class JobConfig {
         }
         b.addTaskConfigs(taskConfigs);
       }
-
+      if (jobBean.jobType != null) {
+        b.setJobType(jobBean.jobType);
+      }
       return b;
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/be78a4f1/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index cbd22cd..ddd37d5 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -56,7 +56,8 @@ public class WorkflowConfig {
     Terminable,
     FailureThreshold,
     /* this is for non-terminable workflow. */
-    capacity
+    capacity,
+    WorkflowType
   }
 
   /* Default values */
@@ -77,9 +78,11 @@ public class WorkflowConfig {
   private final ScheduleConfig _scheduleConfig;
   private final int _failureThreshold;
   private final int _capacity;
+  private final String _workflowType;
 
   protected WorkflowConfig(JobDag jobDag, int parallelJobs, TargetState targetState, long expiry,
-      int failureThreshold, boolean terminable, ScheduleConfig scheduleConfig, int capacity) {
+      int failureThreshold, boolean terminable, ScheduleConfig scheduleConfig, int capacity,
+      String workflowType) {
     _jobDag = jobDag;
     _parallelJobs = parallelJobs;
     _targetState = targetState;
@@ -88,6 +91,7 @@ public class WorkflowConfig {
     _terminable = terminable;
     _scheduleConfig = scheduleConfig;
     _capacity = capacity;
+    _workflowType = workflowType;
   }
 
   public JobDag getJobDag() {
@@ -117,6 +121,10 @@ public class WorkflowConfig {
    */
   public int getCapacity() { return _capacity; }
 
+  public String getWorkflowType() {
+    return _workflowType;
+  }
+
   public boolean isTerminable() {
     return _terminable;
   }
@@ -184,6 +192,9 @@ public class WorkflowConfig {
             scheduleConfig.getRecurrenceInterval().toString());
       }
     }
+    if (_workflowType != null) {
+      cfgMap.put(WorkflowConfigProperty.WorkflowType.name(), _workflowType);
+    }
     return cfgMap;
   }
 
@@ -238,12 +249,13 @@ public class WorkflowConfig {
     private boolean _isTerminable = true;
     private int _capacity = Integer.MAX_VALUE;
     private ScheduleConfig _scheduleConfig;
+    private String _workflowType;
 
     public WorkflowConfig build() {
       validate();
 
       return new WorkflowConfig(_taskDag, _parallelJobs, _targetState, _expiry, _failureThreshold,
-          _isTerminable, _scheduleConfig, _capacity);
+          _isTerminable, _scheduleConfig, _capacity, _workflowType);
     }
 
     public Builder() {}
@@ -289,6 +301,11 @@ public class WorkflowConfig {
       return this;
     }
 
+    public Builder setWorkFlowType(String workflowType) {
+      _workflowType = workflowType;
+      return this;
+    }
+
     public Builder setTerminable(boolean isTerminable) {
       _isTerminable = isTerminable;
       return this;
@@ -355,6 +372,10 @@ public class WorkflowConfig {
       if (scheduleConfig != null) {
         setScheduleConfig(scheduleConfig);
       }
+
+      if (cfg.containsKey(WorkflowConfigProperty.WorkflowType.name())) {
+        setWorkFlowType(cfg.get(WorkflowConfigProperty.WorkflowType.name()));
+      }
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/be78a4f1/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java b/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
index a570026..0080cc6 100644
--- a/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
+++ b/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
@@ -31,6 +31,7 @@ public class JobBean {
   public String name;
   public List<String> parents;
   public String targetResource;
+  public String jobType;
   public List<String> targetPartitionStates;
   public List<String> targetPartitions;
   public String command;

http://git-wip-us.apache.org/repos/asf/helix/blob/be78a4f1/helix-core/src/main/java/org/apache/helix/task/beans/WorkflowBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/beans/WorkflowBean.java b/helix-core/src/main/java/org/apache/helix/task/beans/WorkflowBean.java
index a59e818..2a9563e 100644
--- a/helix-core/src/main/java/org/apache/helix/task/beans/WorkflowBean.java
+++ b/helix-core/src/main/java/org/apache/helix/task/beans/WorkflowBean.java
@@ -31,4 +31,5 @@ public class WorkflowBean {
   public List<JobBean> jobs;
   public ScheduleBean schedule;
   public long expiry = WorkflowConfig.DEFAULT_EXPIRY;
+  public String workflowType;
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/be78a4f1/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
new file mode 100644
index 0000000..0b02085
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
@@ -0,0 +1,60 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.Workflow;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestJobAndWorkflowType extends TaskTestBase {
+  private static final Logger LOG = Logger.getLogger(TestJobAndWorkflowType.class);
+
+  @Test
+  public void testJobAndWorkflowType() throws InterruptedException {
+    LOG.info("Start testing job and workflow type");
+    String jobName = TestHelper.getTestMethodName();
+    JobConfig.Builder jobConfig = JobConfig.Builder.fromMap(WorkflowGenerator.DEFAULT_JOB_CONFIG)
+        .setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG).setJobType("JobTestType");
+
+    Map<String, String> tmp = new HashMap<String, String>();
+    tmp.put("WorkflowType", "WorkflowTestType");
+    Workflow.Builder builder =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobName, jobConfig).fromMap(tmp);
+
+    // Start workflow
+    _driver.start(builder.build());
+
+    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    String fetchedJobType =
+        _driver.getJobConfig(String.format("%s_%s", jobName, jobName)).getJobType();
+    String fetchedWorkflowType =
+        _driver.getWorkflowConfig(jobName).getWorkflowType();
+
+    Assert.assertEquals(fetchedJobType, "JobTestType");
+    Assert.assertEquals(fetchedWorkflowType, "WorkflowTestType");
+  }
+}


[17/33] helix git commit: Added method to TaskDriver to get all workflows from a cluster. Added methods to convert HelixProperty to WorkflowConfig and JobConfig.

Posted by lx...@apache.org.
Added method to TaskDriver to get all workflows from a cluster. Added methods to convert HelixProperty to WorkflowConfig and JobConfig.


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

Branch: refs/heads/helix-0.6.x
Commit: 9f802064fbc782fd6820d551bb1cf39ba4c53108
Parents: ead8301
Author: Lei Xia <lx...@linkedin.com>
Authored: Fri Apr 8 10:52:23 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 15:01:06 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobConfig.java   |  8 +++++++
 .../java/org/apache/helix/task/TaskDriver.java  | 22 ++++++++++++++++++++
 .../org/apache/helix/task/WorkflowConfig.java   | 13 +++++++++++-
 3 files changed, 42 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9f802064/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index 4d5aa94..ac086d3 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -34,10 +34,12 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.helix.task.beans.JobBean;
 import org.apache.helix.task.beans.TaskBean;
+import org.apache.helix.HelixProperty;
 
 /**
  * Provides a typed interface to job configurations.
  */
+// TODO: extends JobConfig from ResourceConfig
 public class JobConfig {
 
   /**
@@ -268,6 +270,12 @@ public class JobConfig {
     return cfgMap;
   }
 
+  public static JobConfig fromHelixProperty(HelixProperty property)
+      throws IllegalArgumentException {
+    Map<String, String> configs = property.getRecord().getSimpleFields();
+    return Builder.fromMap(configs).build();
+  }
+
   /**
    * A builder for {@link JobConfig}. Validates the configurations.
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/9f802064/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index c0d7852..b3a0364 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -749,6 +749,28 @@ public class TaskDriver {
     return TaskUtil.getJobCfg(manager, job);
   }
 
+  /**
+   * Batch get the configurations of all workflows in this cluster.
+   *
+   * @return
+   */
+  public Map<String, WorkflowConfig> getWorkflows() {
+    Map<String, WorkflowConfig> workflowConfigMap = new HashMap<String, WorkflowConfig>();
+    Map<String, ResourceConfig> resourceConfigMap =
+        _accessor.getChildValuesMap(_accessor.keyBuilder().resourceConfigs());
+
+    for (Map.Entry<String, ResourceConfig> resource : resourceConfigMap.entrySet()) {
+      try {
+        WorkflowConfig config = WorkflowConfig.fromHelixProperty(resource.getValue());
+        workflowConfigMap.put(resource.getKey(), config);
+      } catch (IllegalArgumentException ex) {
+        // ignore if it is not a workflow config.
+      }
+    }
+
+    return workflowConfigMap;
+  }
+
   public void list(String resource) {
     WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_accessor, resource);
     if (wCfg == null) {

http://git-wip-us.apache.org/repos/asf/helix/blob/9f802064/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index 844bdf0..cbd22cd 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -28,11 +28,13 @@ import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.HelixException;
 import org.apache.helix.task.beans.WorkflowBean;
+import org.apache.helix.HelixProperty;
 import org.apache.log4j.Logger;
 
 /**
  * Provides a typed interface to workflow level configurations. Validates the configurations.
  */
+// TODO: extends WorkflowConfig from ResourceConfig
 public class WorkflowConfig {
   private static final Logger LOG = Logger.getLogger(WorkflowConfig.class);
 
@@ -217,6 +219,16 @@ public class WorkflowConfig {
     return null;
   }
 
+  public static WorkflowConfig fromHelixProperty(HelixProperty property)
+      throws IllegalArgumentException {
+    Map<String, String> configs = property.getRecord().getSimpleFields();
+    if (!configs.containsKey(WorkflowConfigProperty.Dag.name())) {
+      throw new IllegalArgumentException(
+          String.format("%s is an invalid WorkflowConfig", property.getId()));
+    }
+    return Builder.fromMap(configs).build();
+  }
+
   public static class Builder {
     private JobDag _taskDag = JobDag.EMPTY_DAG;
     private int _parallelJobs = 1;
@@ -399,5 +411,4 @@ public class WorkflowConfig {
       }
     }
   }
-
 }


[08/33] helix git commit: Add new job option to allow contining a job even its direct dependent job fails.

Posted by lx...@apache.org.
Add new job option to allow contining a job even its direct dependent job fails.


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

Branch: refs/heads/helix-0.6.x
Commit: be660245fc1a9f4b22fba58c4b25a1af19555066
Parents: 579d82f
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Jan 27 10:10:31 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:44:56 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobConfig.java   |  26 +-
 .../org/apache/helix/task/JobRebalancer.java    |  26 +-
 .../org/apache/helix/task/TaskRebalancer.java   |  79 +++++-
 .../org/apache/helix/task/WorkflowConfig.java   |  23 +-
 .../apache/helix/task/WorkflowRebalancer.java   |  28 +-
 .../apache/helix/integration/task/MockTask.java |   2 +-
 .../helix/integration/task/TaskTestUtil.java    |   7 +-
 .../task/TestJobFailureDependence.java          | 283 +++++++++++++++++++
 .../task/TestRunJobsWithMissingTarget.java      |  41 ++-
 9 files changed, 469 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index 37a2f35..65a9caf 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -96,6 +96,11 @@ public class JobConfig {
     TASK_RETRY_DELAY("TaskRetryDelay"),
 
     /**
+     * Whether failure of directly dependent jobs should fail this job.
+     */
+    IGNORE_DEPENDENT_JOB_FAILURE("IgnoreDependentJobFailure"),
+
+    /**
      * The individual task configurations, if any *
      */
     TASK_CONFIGS("TaskConfigs"),
@@ -124,6 +129,7 @@ public class JobConfig {
   public static final int DEFAULT_FAILURE_THRESHOLD = 0;
   public static final int DEFAULT_MAX_FORCED_REASSIGNMENTS_PER_TASK = 0;
   public static final boolean DEFAULT_DISABLE_EXTERNALVIEW = false;
+  public static final boolean DEFAULT_IGNORE_DEPENDENT_JOB_FAILURE = false;
 
   private final String _workflow;
   private final String _targetResource;
@@ -138,13 +144,14 @@ public class JobConfig {
   private final int _failureThreshold;
   private final long _retryDelay;
   private final boolean _disableExternalView;
+  private final boolean _ignoreDependentJobFailure;
   private final Map<String, TaskConfig> _taskConfigMap;
 
   private JobConfig(String workflow, String targetResource, List<String> targetPartitions,
       Set<String> targetPartitionStates, String command, Map<String, String> jobCommandConfigMap,
       long timeoutPerTask, int numConcurrentTasksPerInstance, int maxAttemptsPerTask,
       int maxForcedReassignmentsPerTask, int failureThreshold, long retryDelay,
-      boolean disableExternalView, Map<String, TaskConfig> taskConfigMap) {
+      boolean disableExternalView, boolean ignoreDependentJobFailure, Map<String, TaskConfig> taskConfigMap) {
     _workflow = workflow;
     _targetResource = targetResource;
     _targetPartitions = targetPartitions;
@@ -158,6 +165,7 @@ public class JobConfig {
     _failureThreshold = failureThreshold;
     _retryDelay = retryDelay;
     _disableExternalView = disableExternalView;
+    _ignoreDependentJobFailure = ignoreDependentJobFailure;
     if (taskConfigMap != null) {
       _taskConfigMap = taskConfigMap;
     } else {
@@ -217,6 +225,8 @@ public class JobConfig {
     return _disableExternalView;
   }
 
+  public boolean isIgnoreDependentJobFailure() { return _ignoreDependentJobFailure; }
+
   public Map<String, TaskConfig> getTaskConfigMap() {
     return _taskConfigMap;
   }
@@ -260,6 +270,8 @@ public class JobConfig {
         Boolean.toString(_disableExternalView));
     cfgMap.put(JobConfigProperty.NUM_CONCURRENT_TASKS_PER_INSTANCE.value(),
         "" + _numConcurrentTasksPerInstance);
+    cfgMap.put(JobConfigProperty.IGNORE_DEPENDENT_JOB_FAILURE.value(),
+        Boolean.toString(_ignoreDependentJobFailure));
     return cfgMap;
   }
 
@@ -281,6 +293,7 @@ public class JobConfig {
     private int _failureThreshold = DEFAULT_FAILURE_THRESHOLD;
     private long _retryDelay = DEFAULT_TASK_RETRY_DELAY;
     private boolean _disableExternalView = DEFAULT_DISABLE_EXTERNALVIEW;
+    private boolean _ignoreDependentJobFailure = DEFAULT_IGNORE_DEPENDENT_JOB_FAILURE;
 
     public JobConfig build() {
       validate();
@@ -288,7 +301,7 @@ public class JobConfig {
       return new JobConfig(_workflow, _targetResource, _targetPartitions, _targetPartitionStates,
           _command, _commandConfig, _timeoutPerTask, _numConcurrentTasksPerInstance,
           _maxAttemptsPerTask, _maxForcedReassignmentsPerTask, _failureThreshold, _retryDelay,
-          _disableExternalView, _taskConfigMap);
+          _disableExternalView, _ignoreDependentJobFailure, _taskConfigMap);
     }
 
     /**
@@ -346,6 +359,10 @@ public class JobConfig {
         b.setDisableExternalView(
             Boolean.valueOf(cfg.get(JobConfigProperty.DISABLE_EXTERNALVIEW.value())));
       }
+      if (cfg.containsKey(JobConfigProperty.IGNORE_DEPENDENT_JOB_FAILURE.value())) {
+        b.setIgnoreDependentJobFailure(
+            Boolean.valueOf(cfg.get(JobConfigProperty.IGNORE_DEPENDENT_JOB_FAILURE.value())));
+      }
       return b;
     }
 
@@ -414,6 +431,11 @@ public class JobConfig {
       return this;
     }
 
+    public Builder setIgnoreDependentJobFailure(boolean ignoreDependentJobFailure) {
+      _ignoreDependentJobFailure = ignoreDependentJobFailure;
+      return this;
+    }
+
     public Builder addTaskConfigs(List<TaskConfig> taskConfigs) {
       if (taskConfigs != null) {
         for (TaskConfig taskConfig : taskConfigs) {

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index 7eeafc7..5b41773 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -86,22 +86,28 @@ public class JobRebalancer extends TaskRebalancer {
       return buildEmptyAssignment(jobName, currStateOutput);
     }
 
+    // Stop current run of the job if workflow or job is already in final state (failed or completed)
+    TaskState workflowState = workflowCtx.getWorkflowState();
     TaskState jobState = workflowCtx.getJobState(jobName);
     // The job is already in a final state (completed/failed).
-    if (jobState == TaskState.FAILED || jobState == TaskState.COMPLETED) {
-      LOG.info("Job " + jobName + " is failed or already completed, clean up IS.");
+    if (workflowState == TaskState.FAILED || workflowState == TaskState.COMPLETED ||
+        jobState == TaskState.FAILED || jobState == TaskState.COMPLETED) {
+      LOG.info(String.format(
+          "Workflow %s or job %s is already failed or completed, workflow state (%s), job state (%s), clean up job IS.",
+          workflowResource, jobName, workflowState, jobState));
       cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobName);
       _scheduledRebalancer.removeScheduledRebalance(jobName);
       return buildEmptyAssignment(jobName, currStateOutput);
     }
 
     if (!isWorkflowReadyForSchedule(workflowCfg)) {
-      LOG.info("Job is not ready to be scheduled since workflow is not ready " + jobName);
+      LOG.info("Job is not ready to be run since workflow is not ready " + jobName);
       return buildEmptyAssignment(jobName, currStateOutput);
     }
 
-    if (!isJobReadyToSchedule(jobName, workflowCfg, workflowCtx)) {
-      LOG.info("Job is not ready to be scheduled " + jobName);
+    if (!isJobStarted(jobName, workflowCtx) && !isJobReadyToSchedule(jobName, workflowCfg,
+        workflowCtx)) {
+      LOG.info("Job is not ready to run " + jobName);
       return buildEmptyAssignment(jobName, currStateOutput);
     }
 
@@ -429,16 +435,6 @@ public class JobRebalancer extends TaskRebalancer {
     return ra;
   }
 
-  private void markJobFailed(String jobName, JobContext jobContext, WorkflowConfig workflowConfig,
-      WorkflowContext workflowContext) {
-    long currentTime = System.currentTimeMillis();
-    workflowContext.setJobState(jobName, TaskState.FAILED);
-    jobContext.setFinishTime(currentTime);
-    if (isWorkflowFinished(workflowContext, workflowConfig)) {
-      workflowContext.setFinishTime(currentTime);
-    }
-  }
-
   private void markJobComplete(String jobName, JobContext jobContext,
       WorkflowConfig workflowConfig, WorkflowContext workflowContext) {
     long currentTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
index b006efc..6aaeb5f 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
@@ -74,13 +74,17 @@ public abstract class TaskRebalancer implements Rebalancer, MappingCalculator {
    */
   protected boolean isWorkflowFinished(WorkflowContext ctx, WorkflowConfig cfg) {
     boolean incomplete = false;
+    int failedJobs = 0;
     for (String job : cfg.getJobDag().getAllNodes()) {
       TaskState jobState = ctx.getJobState(job);
       if (jobState == TaskState.FAILED) {
-        ctx.setWorkflowState(TaskState.FAILED);
-        return true;
+        failedJobs ++;
+        if (failedJobs > cfg.getFailureThreshold()) {
+          ctx.setWorkflowState(TaskState.FAILED);
+          return true;
+        }
       }
-      if (jobState != TaskState.COMPLETED) {
+      if (jobState != TaskState.COMPLETED && jobState != TaskState.FAILED) {
         incomplete = true;
       }
     }
@@ -136,31 +140,78 @@ public abstract class TaskRebalancer implements Rebalancer, MappingCalculator {
   protected boolean isJobReadyToSchedule(String job, WorkflowConfig workflowCfg,
       WorkflowContext workflowCtx) {
     int notStartedCount = 0;
-    int inCompleteCount = 0;
     int failedCount = 0;
 
-    for (String ancestor : workflowCfg.getJobDag().getAncestors(job)) {
-      TaskState jobState = workflowCtx.getJobState(ancestor);
+    for (String parent : workflowCfg.getJobDag().getDirectParents(job)) {
+      TaskState jobState = workflowCtx.getJobState(parent);
       if (jobState == null || jobState == TaskState.NOT_STARTED) {
         ++notStartedCount;
-      } else if (jobState == TaskState.IN_PROGRESS || jobState == TaskState.STOPPED) {
-        ++inCompleteCount;
-      } else if (jobState == TaskState.FAILED) {
+      }
+      if (jobState == TaskState.FAILED) {
         ++failedCount;
       }
     }
 
-    if (notStartedCount > 0 || inCompleteCount >= workflowCfg.getParallelJobs()
-        || failedCount > 0) {
-      LOG.debug(String.format(
-          "Job %s is not ready to start, notStartedParent(s)=%d, inCompleteParent(s)=%d, failedParent(s)=%d.",
-          job, notStartedCount, inCompleteCount, failedCount));
+    if (notStartedCount > 0) {
+      LOG.debug(String
+          .format("Job %s is not ready to start, notStartedParent(s)=%d.", job, notStartedCount));
+      return false;
+    }
+
+    JobConfig jobConfig = TaskUtil.getJobCfg(_manager, job);
+    if (failedCount > 0 && !jobConfig.isIgnoreDependentJobFailure()) {
+      markJobFailed(job, null, workflowCfg, workflowCtx);
+      LOG.debug(
+          String.format("Job %s is not ready to start, failedCount(s)=%d.", job, failedCount));
+      return false;
+    }
+
+    int inCompleteCount = getInCompleteJobCount(workflowCfg, workflowCtx);
+    if (inCompleteCount >= workflowCfg.getParallelJobs()) {
+      LOG.debug(String
+          .format("Job %s is not ready to schedule, inCompleteJobs(s)=%d.", job, inCompleteCount));
       return false;
     }
 
     return true;
   }
 
+  protected boolean isJobStarted(String job, WorkflowContext workflowContext) {
+    TaskState jobState = workflowContext.getJobState(job);
+    return (jobState != null && jobState != TaskState.NOT_STARTED);
+  }
+
+  /**
+   * Count the number of jobs in a workflow that are in progress.
+   *
+   * @param workflowCfg
+   * @param workflowCtx
+   * @return
+   */
+  protected int getInCompleteJobCount(WorkflowConfig workflowCfg, WorkflowContext workflowCtx) {
+    int inCompleteCount = 0;
+    for (String jobName : workflowCfg.getJobDag().getAllNodes()) {
+      TaskState jobState = workflowCtx.getJobState(jobName);
+      if (jobState == TaskState.IN_PROGRESS || jobState == TaskState.STOPPED) {
+        ++inCompleteCount;
+      }
+    }
+
+    return inCompleteCount;
+  }
+
+  protected void markJobFailed(String jobName, JobContext jobContext, WorkflowConfig workflowConfig,
+      WorkflowContext workflowContext) {
+    long currentTime = System.currentTimeMillis();
+    workflowContext.setJobState(jobName, TaskState.FAILED);
+    if (jobContext != null) {
+      jobContext.setFinishTime(currentTime);
+    }
+    if (isWorkflowFinished(workflowContext, workflowConfig)) {
+      workflowContext.setFinishTime(currentTime);
+    }
+  }
+
   /**
    * Check if a workflow is ready to schedule.
    *

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index 4c81654..955cb77 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -40,9 +40,11 @@ public class WorkflowConfig {
   public static final String RECURRENCE_UNIT = "RecurrenceUnit";
   public static final String RECURRENCE_INTERVAL = "RecurrenceInterval";
   public static final String TERMINABLE = "Terminable";
+  public static final String FAILURE_THRESHOLD = "FailureThreshold";
 
   /* Default values */
   public static final long DEFAULT_EXPIRY = 24 * 60 * 60 * 1000;
+  public static final int DEFAULT_FAILURE_THRESHOLD = 0;
 
   /* Member variables */
   // TODO: jobDag should not be in the workflowConfig.
@@ -56,13 +58,15 @@ public class WorkflowConfig {
   private final long _expiry;
   private final boolean _terminable;
   private final ScheduleConfig _scheduleConfig;
+  private final int _failureThreshold;
 
   protected WorkflowConfig(JobDag jobDag, int parallelJobs, TargetState targetState, long expiry,
-      boolean terminable, ScheduleConfig scheduleConfig) {
+      int failureThreshold, boolean terminable, ScheduleConfig scheduleConfig) {
     _jobDag = jobDag;
     _parallelJobs = parallelJobs;
     _targetState = targetState;
     _expiry = expiry;
+    _failureThreshold = failureThreshold;
     _terminable = terminable;
     _scheduleConfig = scheduleConfig;
   }
@@ -83,6 +87,10 @@ public class WorkflowConfig {
     return _expiry;
   }
 
+  public int getFailureThreshold() {
+    return _failureThreshold;
+  }
+
   public boolean isTerminable() {
     return _terminable;
   }
@@ -128,6 +136,7 @@ public class WorkflowConfig {
     cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(getExpiry()));
     cfgMap.put(WorkflowConfig.TARGET_STATE, getTargetState().name());
     cfgMap.put(WorkflowConfig.TERMINABLE, String.valueOf(isTerminable()));
+    cfgMap.put(WorkflowConfig.FAILURE_THRESHOLD, String.valueOf(getFailureThreshold()));
 
     // Populate schedule if present
     ScheduleConfig scheduleConfig = getScheduleConfig();
@@ -151,13 +160,15 @@ public class WorkflowConfig {
     private int _parallelJobs = 1;
     private TargetState _targetState = TargetState.START;
     private long _expiry = DEFAULT_EXPIRY;
+    private int _failureThreshold = DEFAULT_FAILURE_THRESHOLD;
     private boolean _isTerminable = true;
     private ScheduleConfig _scheduleConfig;
 
     public WorkflowConfig build() {
       validate();
 
-      return new WorkflowConfig(_taskDag, _parallelJobs, _targetState, _expiry, _isTerminable, _scheduleConfig);
+      return new WorkflowConfig(_taskDag, _parallelJobs, _targetState, _expiry, _failureThreshold,
+          _isTerminable, _scheduleConfig);
     }
 
     public Builder() {}
@@ -191,6 +202,11 @@ public class WorkflowConfig {
       return this;
     }
 
+    public Builder setFailureThreshold(int failureThreshold) {
+      _failureThreshold = failureThreshold;
+      return this;
+    }
+
     public Builder setTerminable(boolean isTerminable) {
       _isTerminable = isTerminable;
       return this;
@@ -211,6 +227,9 @@ public class WorkflowConfig {
       if (cfg.containsKey(EXPIRY)) {
         b.setExpiry(Long.parseLong(cfg.get(EXPIRY)));
       }
+      if (cfg.containsKey(FAILURE_THRESHOLD)) {
+        b.setFailureThreshold(Integer.parseInt(cfg.get(FAILURE_THRESHOLD)));
+      }
       if (cfg.containsKey(DAG)) {
         b.setJobDag(JobDag.fromJson(cfg.get(DAG)));
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
index 05b6dc6..682ac77 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
@@ -30,7 +30,13 @@ import org.apache.log4j.Logger;
 
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TimeZone;
 
 /**
  * Custom rebalancer implementation for the {@code Workflow} in task state model.
@@ -63,7 +69,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
     TargetState targetState = workflowCfg.getTargetState();
     if (targetState == TargetState.DELETE) {
       LOG.info("Workflow is marked as deleted " + workflow + " cleaning up the workflow context.");
-      cleanupWorkflow(workflow, workflowCfg, workflowCtx);
+      cleanupWorkflow(workflow, workflowCfg);
       return buildEmptyAssignment(workflow, currStateOutput);
     }
 
@@ -91,7 +97,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
       // Check if this workflow has been finished past its expiry.
       if (workflowCtx.getFinishTime() + expiryTime <= currentTime) {
         LOG.info("Workflow " + workflow + " passed expiry time, cleaning up the workflow context.");
-        cleanupWorkflow(workflow, workflowCfg, workflowCtx);
+        cleanupWorkflow(workflow, workflowCfg);
       } else {
         // schedule future cleanup work
         long cleanupTime = workflowCtx.getFinishTime() + expiryTime;
@@ -113,7 +119,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
         scheduleWorkflowIfReady(workflow, workflowCfg, workflowCtx);
     if (isReady) {
       // Schedule jobs from this workflow.
-      scheduleJobs(workflowCfg, workflowCtx);
+      scheduleJobs(workflow, workflowCfg, workflowCtx);
     } else {
       LOG.debug("Workflow " + workflow + " is not ready to be scheduled.");
     }
@@ -126,23 +132,32 @@ public class WorkflowRebalancer extends TaskRebalancer {
    * Figure out whether the jobs in the workflow should be run,
    * and if it's ready, then just schedule it
    */
-  private void scheduleJobs(WorkflowConfig workflowCfg, WorkflowContext workflowCtx) {
+  private void scheduleJobs(String workflow, WorkflowConfig workflowCfg, WorkflowContext workflowCtx) {
     ScheduleConfig scheduleConfig = workflowCfg.getScheduleConfig();
     if (scheduleConfig != null && scheduleConfig.isRecurring()) {
       LOG.debug("Jobs from recurring workflow are not schedule-able");
       return;
     }
 
+    int scheduledJobs = 0;
     for (String job : workflowCfg.getJobDag().getAllNodes()) {
       TaskState jobState = workflowCtx.getJobState(job);
       if (jobState != null && !jobState.equals(TaskState.NOT_STARTED)) {
         LOG.debug("Job " + job + " is already started or completed.");
         continue;
       }
+
+      if (scheduledJobs >= workflowCfg.getParallelJobs()) {
+        LOG.debug(String.format("Workflow %s already have enough job in progress, "
+                + "scheduledJobs(s)=%d, stop scheduling more jobs", workflow, scheduledJobs));
+        break;
+      }
+
       // check ancestor job status
       if (isJobReadyToSchedule(job, workflowCfg, workflowCtx)) {
         JobConfig jobConfig = TaskUtil.getJobCfg(_manager, job);
         scheduleSingleJob(job, jobConfig);
+        scheduledJobs++;
       }
     }
   }
@@ -382,8 +397,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
    * Cleans up workflow configs and workflow contexts associated with this workflow,
    * including all job-level configs and context, plus workflow-level information.
    */
-  private void cleanupWorkflow(String workflow, WorkflowConfig workflowcfg,
-      WorkflowContext workflowCtx) {
+  private void cleanupWorkflow(String workflow, WorkflowConfig workflowcfg) {
     LOG.info("Cleaning up workflow: " + workflow);
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
index 71fa12d..dad9949 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
@@ -39,7 +39,7 @@ public class MockTask implements Task {
     if (cfg == null) {
       cfg = Collections.emptyMap();
     }
-    _delay = cfg.containsKey(TIMEOUT_CONFIG) ? Long.parseLong(cfg.get(TIMEOUT_CONFIG)) : 200L;
+    _delay = cfg.containsKey(TIMEOUT_CONFIG) ? Long.parseLong(cfg.get(TIMEOUT_CONFIG)) : 100L;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index 9796497..011f532 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -258,7 +258,7 @@ public class TaskTestUtil {
     return buildRecurrentJobQueue(jobQueueName, 0);
   }
 
-  public static JobQueue.Builder buildJobQueue(String jobQueueName, int delayStart) {
+  public static JobQueue.Builder buildJobQueue(String jobQueueName, int delayStart, int failureThreshold) {
     Map<String, String> cfgMap = new HashMap<String, String>();
     cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(120000));
     Calendar cal = Calendar.getInstance();
@@ -267,10 +267,13 @@ public class TaskTestUtil {
     cal.set(Calendar.MILLISECOND, 0);
     cfgMap.put(WorkflowConfig.START_TIME,
         WorkflowConfig.getDefaultDateFormat().format(cal.getTime()));
+    if (failureThreshold > 0) {
+      cfgMap.put(WorkflowConfig.FAILURE_THRESHOLD, String.valueOf(failureThreshold));
+    }
     return new JobQueue.Builder(jobQueueName).fromMap(cfgMap);
   }
 
   public static JobQueue.Builder buildJobQueue(String jobQueueName) {
-    return buildJobQueue(jobQueueName, 0);
+    return buildJobQueue(jobQueueName, 0, 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
new file mode 100644
index 0000000..9e2456c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
@@ -0,0 +1,283 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import com.google.common.collect.Sets;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.TestHelper;
+import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.Task;
+import org.apache.helix.task.TaskCallbackContext;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.WorkflowConfig;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestJobFailureDependence extends ZkIntegrationTestBase {
+  private static final Logger LOG = Logger.getLogger(TestJobFailureDependence.class);
+  private static final int num_nodes = 5;
+  private static final int num_dbs = 5;
+  private static final int START_PORT = 12918;
+  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
+  private static final int NUM_PARTITIONS = 20;
+  private static final int NUM_REPLICAS = 3;
+  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
+  private final MockParticipantManager[] _participants = new MockParticipantManager[num_nodes];
+  private ClusterControllerManager _controller;
+  private ClusterSetup _setupTool;
+
+  private List<String> _test_dbs = new ArrayList<String>();
+
+  private HelixManager _manager;
+  private TaskDriver _driver;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+
+    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool.addCluster(CLUSTER_NAME, true);
+    for (int i = 0; i < num_nodes; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    }
+
+    // Set up target dbs
+    for (int i = 0; i < num_dbs; i++) {
+      String db = "TestDB" + i;
+      _setupTool
+          .addResourceToCluster(CLUSTER_NAME, db, NUM_PARTITIONS + 10 * i, MASTER_SLAVE_STATE_MODEL,
+              IdealState.RebalanceMode.FULL_AUTO.toString());
+      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, NUM_REPLICAS);
+      _test_dbs.add(db);
+    }
+
+    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
+    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
+      @Override public Task createNewTask(TaskCallbackContext context) {
+        return new MockTask(context);
+      }
+    });
+
+    // start dummy participants
+    for (int i = 0; i < num_nodes; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+
+      // Register a Task state model factory.
+      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
+      stateMachine.registerStateModelFactory("Task",
+          new TaskStateModelFactory(_participants[i], taskFactoryReg));
+
+      _participants[i].syncStart();
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // create cluster manager
+    _manager = HelixManagerFactory
+        .getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR, ZK_ADDR);
+    _manager.connect();
+
+    _driver = new TaskDriver(_manager);
+
+    boolean result = ClusterStateVerifier.verifyByZkCallback(
+        new ClusterStateVerifier.MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+
+    result = ClusterStateVerifier.verifyByZkCallback(
+        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    _manager.disconnect();
+    _controller.syncStop();
+    for (int i = 0; i < num_nodes; i++) {
+      _participants[i].syncStop();
+    }
+  }
+
+  @Test
+  public void testJobDependantFailure() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 100);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i < num_dbs; i++) {
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+              .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
+      String jobName = "job" + _test_dbs.get(i);
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuilder.build());
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+
+    // all jobs after failed job should fail too.
+    for (int i = 2; i < num_dbs; i++) {
+      String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(i));
+      TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.FAILED);
+    }
+  }
+
+  @Test
+  public void testJobDependantWorkflowFailure() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i < num_dbs; i++) {
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+              .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
+      String jobName = "job" + _test_dbs.get(i);
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuilder.build());
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+
+    String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(2));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+  }
+
+  @Test
+  public void testIgnoreJobDependantFailure() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 100);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i < num_dbs; i++) {
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+              .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
+      String jobName = "job" + _test_dbs.get(i);
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuilder.build());
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+    String namedSpaceJob2 = String.format("%s_%s", queueName, currentJobNames.get(2));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob2, TaskState.FAILED);
+
+    // all jobs after failed job should complete.
+    for (int i = 3; i < num_dbs; i++) {
+      String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(i));
+      TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.COMPLETED);
+    }
+  }
+
+  @Test
+  public void testWorkflowFailureJobThreshold() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 3);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i < num_dbs; i++) {
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+              .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
+      String jobName = "job" + _test_dbs.get(i);
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuilder.build());
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(1));
+
+    String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    String lastJob =
+        String.format("%s_%s", queueName, currentJobNames.get(currentJobNames.size() - 1));
+    TaskTestUtil.pollForJobState(_driver, queueName, lastJob, TaskState.COMPLETED);
+
+    _driver.flushQueue(queueName);
+
+    WorkflowConfig currentWorkflowConfig = _driver.getWorkflowConfig(queueName);
+    WorkflowConfig.Builder configBuilder = new WorkflowConfig.Builder(currentWorkflowConfig);
+
+    configBuilder.setFailureThreshold(0);
+    _driver.updateWorkflow(queueName, configBuilder.build());
+    _driver.stop(queueName);
+
+    for (int i = 0; i < num_dbs; i++) {
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+              .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
+      String jobName = "job" + _test_dbs.get(i);
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      _driver.enqueueJob(queueName, jobName, jobConfig);
+    }
+
+    _driver.resume(queueName);
+
+    namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/helix/blob/be660245/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index 101604b..7eeb3f4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -140,6 +140,7 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     for (int i = 0; i < num_nodes; i++) {
       _participants[i].syncStop();
     }
+    _setupTool.deleteCluster(CLUSTER_NAME);
   }
 
   @Test
@@ -161,12 +162,44 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
       currentJobNames.add(jobName);
     }
 
-    _driver.start(queueBuilder.build());
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+    _driver.start(queueBuilder.build());
 
-    String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(2));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(2));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.FAILED);
     TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+
+    _driver.delete(queueName);
+  }
+
+  @Test
+  public void testJobContinueUponParentJobFailure() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 3);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i < num_dbs; i++) {
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+              .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
+      String jobName = "job" + _test_dbs.get(i);
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuilder.build());
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(1));
+
+    String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    String lastJob =
+        String.format("%s_%s", queueName, currentJobNames.get(currentJobNames.size() - 1));
+    TaskTestUtil.pollForJobState(_driver, queueName, lastJob, TaskState.COMPLETED);
+
+    _driver.delete(queueName);
   }
 
   @Test
@@ -193,5 +226,7 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(0));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
     TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+
+    _driver.delete(queueName);
   }
 }


[30/33] helix git commit: Fix task assignment in instance group tag check

Posted by lx...@apache.org.
Fix task assignment in instance group tag check

1. Fixed task assignment with instacnce group. When target state is not set, it ont check the state of instance.
2. Add a new test to assign task. The instance will be assign to has been disabled. So the task should be hanging there.


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

Branch: refs/heads/helix-0.6.x
Commit: 9a30df46bc054514a708b5070f166ee5aa2e7cd1
Parents: 9508a1a
Author: Junkai Xue <jx...@linkedin.com>
Authored: Fri May 6 12:05:30 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:19:11 2016 -0700

----------------------------------------------------------------------
 .../FixedTargetTaskAssignmentCalculator.java    |  3 +-
 .../integration/task/TestTaskAssignment.java    | 62 ++++++++++++++++++++
 .../task/TestTaskWithInstanceDisabled.java      |  4 +-
 3 files changed, 65 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9a30df46/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
index 0a2e8c5..09db616 100644
--- a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
+++ b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
@@ -168,8 +168,7 @@ public class FixedTargetTaskAssignmentCalculator extends TaskAssignmentCalculato
           String s =
               currStateOutput.getCurrentState(tgtIs.getResourceName(), new Partition(pName),
                   instance);
-          String state = (s == null ? null : s.toString());
-          if (tgtStates == null || tgtStates.contains(state)) {
+          if (s != null && (tgtStates == null || tgtStates.contains(s))) {
             result.get(instance).add(pId);
           }
         }

http://git-wip-us.apache.org/repos/asf/helix/blob/9a30df46/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
new file mode 100644
index 0000000..df976b1
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
@@ -0,0 +1,62 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestTaskAssignment extends TaskTestBase {
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numDbs = 1;
+    _numNodes = 2;
+    _numParitions = 1;
+    _numReplicas = 1;
+    _instanceGroupTag = true;
+    super.beforeClass();
+  }
+
+  @Test
+  public void testTaskAssignment() throws InterruptedException {
+    _setupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, PARTICIPANT_PREFIX + "_" + (_startPort + 0), false);
+    String jobResource = TestHelper.getTestMethodName();
+    JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB);
+
+    Workflow flow =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
+    _driver.start(flow);
+
+    // Wait 1 sec. The task should not be complete since it is not assigned.
+    Thread.sleep(1000L);
+
+    // The task is not assigned so the task state should be null in this case.
+    Assert.assertNull(
+        _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource)).getPartitionState(0));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/9a30df46/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
index 1c5bd36..84e5168 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
@@ -36,7 +36,7 @@ public class TestTaskWithInstanceDisabled extends TaskTestBase {
     _numDbs = 1;
     _numNodes = 2;
     _numParitions = 1;
-    _numReplicas = 1;
+    _numReplicas = 2;
     _partitionVary = false;
     super.beforeClass();
   }
@@ -51,7 +51,7 @@ public class TestTaskWithInstanceDisabled extends TaskTestBase {
         WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
     _driver.start(flow);
 
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED, TaskState.FAILED);
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     Assert.assertEquals(ctx.getAssignedParticipant(0), PARTICIPANT_PREFIX + "_" + (_startPort + 1));
   }


[03/33] helix git commit: Fix the statemodelFactories in Example Process.

Posted by lx...@apache.org.
Fix the statemodelFactories in Example Process.


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

Branch: refs/heads/helix-0.6.x
Commit: 37819298939b61d8f7a589a3404fd3414fbdbd06
Parents: 03b9001
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Jan 27 18:04:23 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:34:37 2016 -0700

----------------------------------------------------------------------
 .../apache/helix/examples/ExampleProcess.java   |  6 +-
 .../LeaderStandbyStateModelFactory.java         | 69 ++++++++++++++++++--
 .../examples/MasterSlaveStateModelFactory.java  |  7 +-
 .../OnlineOfflineStateModelFactory.java         | 37 +++++++++--
 4 files changed, 106 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/37819298/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java b/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java
index 840a963..9328cf4 100644
--- a/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java
+++ b/helix-core/src/main/java/org/apache/helix/examples/ExampleProcess.java
@@ -77,11 +77,11 @@ public class ExampleProcess {
             zkConnectString);
 
     if ("MasterSlave".equalsIgnoreCase(stateModelType)) {
-      stateModelFactory = new MasterSlaveStateModelFactory(delay);
+      stateModelFactory = new MasterSlaveStateModelFactory(this.instanceName, delay);
     } else if ("OnlineOffline".equalsIgnoreCase(stateModelType)) {
-      stateModelFactory = new OnlineOfflineStateModelFactory(delay);
+      stateModelFactory = new OnlineOfflineStateModelFactory(this.instanceName, delay);
     } else if ("LeaderStandby".equalsIgnoreCase(stateModelType)) {
-      stateModelFactory = new LeaderStandbyStateModelFactory(delay);
+      stateModelFactory = new LeaderStandbyStateModelFactory(this.instanceName, delay);
     }
     // genericStateMachineHandler = new StateMachineEngine();
     // genericStateMachineHandler.registerStateModelFactory(stateModelType,

http://git-wip-us.apache.org/repos/asf/helix/blob/37819298/helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java b/helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java
index c0fff7c..a2ca091 100644
--- a/helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/examples/LeaderStandbyStateModelFactory.java
@@ -26,35 +26,96 @@ import org.apache.helix.participant.statemachine.StateModelFactory;
 
 public class LeaderStandbyStateModelFactory extends StateModelFactory<StateModel> {
   int _delay;
+  String _instanceName = "";
 
   public LeaderStandbyStateModelFactory(int delay) {
     _delay = delay;
   }
 
-  @Override
-  public StateModel createNewStateModel(String resourceName, String stateUnitKey) {
+  public LeaderStandbyStateModelFactory(String instanceName) {
+    _instanceName = instanceName;
+    _delay = 10;
+  }
+
+  public LeaderStandbyStateModelFactory(String instanceName, int delay) {
+    _instanceName = instanceName;
+    _delay = delay;
+  }
+
+  public LeaderStandbyStateModelFactory() {
+    this(10);
+  }
+
+  public void setInstanceName(String instanceName) {
+    _instanceName = instanceName;
+  }
+
+  @Override public StateModel createNewStateModel(String resourceName, String partitionName) {
     LeaderStandbyStateModel stateModel = new LeaderStandbyStateModel();
     stateModel.setDelay(_delay);
+    stateModel.setInstanceName(_instanceName);
+    stateModel.setPartitionName(partitionName);
     return stateModel;
   }
 
   public static class LeaderStandbyStateModel extends StateModel {
     int _transDelay = 0;
+    String partitionName;
+    String _instanceName = "";
+
+    public String getPartitionName() {
+      return partitionName;
+    }
+
+    public void setPartitionName(String partitionName) {
+      this.partitionName = partitionName;
+    }
+
+    public void setInstanceName(String instanceName) {
+      _instanceName = instanceName;
+    }
+
 
     public void setDelay(int delay) {
       _transDelay = delay > 0 ? delay : 0;
     }
 
     public void onBecomeLeaderFromStandby(Message message, NotificationContext context) {
-      System.out.println("LeaderStandbyStateModel.onBecomeLeaderFromStandby()");
+      System.out.println("LeaderStandbyStateModel.onBecomeLeaderFromStandby():" + _instanceName
+          + " transitioning from " + message.getFromState() + " to " + message.getToState()
+          + " for " + message.getResourceName() + " " + message.getPartitionName());
       sleep();
     }
 
     public void onBecomeStandbyFromLeader(Message message, NotificationContext context) {
-      System.out.println("LeaderStandbyStateModel.onBecomeStandbyFromLeader()");
+      System.out.println("LeaderStandbyStateModel.onBecomeStandbyFromLeader():" + _instanceName
+          + " transitioning from " + message.getFromState() + " to " + message.getToState()
+          + " for " + message.getResourceName() + " " + message.getPartitionName());
       sleep();
     }
 
+    public void onBecomeStandbyFromOffline(Message message, NotificationContext context) {
+      System.out.println("LeaderStandbyStateModel.onBecomeStandbyFromOffline():" + _instanceName
+          + " transitioning from " + message.getFromState() + " to " + message.getToState()
+          + " for " + message.getResourceName() + " " + message.getPartitionName());
+      sleep();
+    }
+
+    public void onBecomeOfflineFromStandby(Message message, NotificationContext context) {
+      System.out.println("LeaderStandbyStateModel.onBecomeOfflineFromStandby():" + _instanceName
+          + " transitioning from " + message.getFromState() + " to " + message.getToState()
+          + " for " + message.getResourceName() + " " + message.getPartitionName());
+      sleep();
+    }
+
+    public void onBecomeDroppedFromOffline(Message message, NotificationContext context) {
+      System.out.println("LeaderStandbyStateModel.onBecomeDroppedFromOffline():" + _instanceName
+          + " transitioning from " + message.getFromState() + " to " + message.getToState()
+          + " for " + message.getResourceName() + " " + message.getPartitionName());
+      sleep();
+
+    }
+
     private void sleep() {
       try {
         Thread.sleep(_transDelay);

http://git-wip-us.apache.org/repos/asf/helix/blob/37819298/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java b/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
index 85d90b2..1d5e790 100644
--- a/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
@@ -27,7 +27,6 @@ import org.apache.helix.participant.statemachine.StateModelFactory;
 @SuppressWarnings("rawtypes")
 public class MasterSlaveStateModelFactory extends StateModelFactory<StateModel> {
   int _delay;
-
   String _instanceName = "";
 
   public MasterSlaveStateModelFactory(int delay) {
@@ -39,6 +38,11 @@ public class MasterSlaveStateModelFactory extends StateModelFactory<StateModel>
     _delay = 10;
   }
 
+  public MasterSlaveStateModelFactory(String instanceName, int delay) {
+    _instanceName = instanceName;
+    _delay = delay;
+  }
+
   public MasterSlaveStateModelFactory() {
     this(10);
   }
@@ -74,7 +78,6 @@ public class MasterSlaveStateModelFactory extends StateModelFactory<StateModel>
     }
 
     public void onBecomeSlaveFromOffline(Message message, NotificationContext context) {
-
       System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to "
           + message.getToState() + " for " + partitionName);
       sleep();

http://git-wip-us.apache.org/repos/asf/helix/blob/37819298/helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java b/helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java
index e11c6d1..d6edb05 100644
--- a/helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/examples/OnlineOfflineStateModelFactory.java
@@ -26,39 +26,68 @@ import org.apache.helix.participant.statemachine.StateModelFactory;
 
 public class OnlineOfflineStateModelFactory extends StateModelFactory<StateModel> {
   int _delay;
+  String _instanceName = "";
 
   public OnlineOfflineStateModelFactory(int delay) {
     _delay = delay;
   }
 
+  public OnlineOfflineStateModelFactory(String instanceName) {
+    _instanceName = instanceName;
+    _delay = 10;
+  }
+
+  public OnlineOfflineStateModelFactory(String instanceName, int delay) {
+    _instanceName = instanceName;
+    _delay = delay;
+  }
+
+  public OnlineOfflineStateModelFactory() {
+    this(10);
+  }
+
   @Override
   public StateModel createNewStateModel(String resourceName, String stateUnitKey) {
     OnlineOfflineStateModel stateModel = new OnlineOfflineStateModel();
     stateModel.setDelay(_delay);
+    stateModel.setInstanceName(_instanceName);
     return stateModel;
   }
 
   public static class OnlineOfflineStateModel extends StateModel {
     int _transDelay = 0;
+    String _instanceName = "";
 
     public void setDelay(int delay) {
       _transDelay = delay > 0 ? delay : 0;
     }
 
+    public void setInstanceName(String instanceName) {
+      _instanceName = instanceName;
+    }
+
     public void onBecomeOnlineFromOffline(Message message, NotificationContext context) {
-      System.out.println("OnlineOfflineStateModel.onBecomeOnlineFromOffline()");
+      System.out.println(
+          "OnlineOfflineStateModelFactory.onBecomeOnlineFromOffline():" + _instanceName
+              + " transitioning from " + message.getFromState() + " to " + message.getToState()
+              + " for " + message.getResourceName() + " " + message.getPartitionName());
       sleep();
     }
 
     public void onBecomeOfflineFromOnline(Message message, NotificationContext context) {
-      System.out.println("OnlineOfflineStateModel.onBecomeOfflineFromOnline()");
+      System.out.println(
+          "OnlineOfflineStateModelFactory.onBecomeOfflineFromOnline():" + _instanceName
+              + " transitioning from " + message.getFromState() + " to " + message.getToState()
+              + " for " + message.getResourceName() + " " + message.getPartitionName());
       sleep();
     }
 
     public void onBecomeDroppedFromOffline(Message message, NotificationContext context) {
-      System.out.println("OnlineOfflineStateModel.onBecomeDroppedFromOffline()");
+      System.out.println(
+          "OnlineOfflineStateModelFactory.onBecomeDroppedFromOffline():" + _instanceName
+              + " transitioning from " + message.getFromState() + " to " + message.getToState()
+              + " for " + message.getResourceName() + " " + message.getPartitionName());
       sleep();
-
     }
 
     private void sleep() {


[20/33] helix git commit: Add job dependency workflow test 1. Add an option to enable number of partion varies in TaskTestBase 2. Add new test for workflow test for parallel jobs testing.

Posted by lx...@apache.org.
Add job dependency workflow test
1. Add an option to enable number of partion varies in TaskTestBase
2. Add new test for workflow test for parallel jobs testing.


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

Branch: refs/heads/helix-0.6.x
Commit: ddc11f9aca9e3e525d517860006e1d5cbb91ddd4
Parents: be78a4f
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Apr 20 16:33:50 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 15:57:59 2016 -0700

----------------------------------------------------------------------
 .../helix/integration/task/TaskTestBase.java    | 24 +++++-
 .../task/TestWorkflowJobDependency.java         | 88 ++++++++++++++++++++
 2 files changed, 111 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ddc11f9a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
index 50baa14..d6cf9bd 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
@@ -1,5 +1,24 @@
 package org.apache.helix.integration.task;
 
+/*
+ * 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.
+ */
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -31,6 +50,8 @@ public class TaskTestBase extends ZkIntegrationTestBase {
   protected int _numReplicas = 3;
   protected int _numDbs = 1;
 
+  protected Boolean _partitionVary = true;
+
   protected ClusterControllerManager _controller;
 
   protected HelixManager _manager;
@@ -60,9 +81,10 @@ public class TaskTestBase extends ZkIntegrationTestBase {
     // Set up target db
     if (_numDbs > 1) {
       for (int i = 0; i < _numDbs; i++) {
+        int varyNum = _partitionVary == true ? 10 * i : 0;
         String db = WorkflowGenerator.DEFAULT_TGT_DB + i;
         _setupTool
-            .addResourceToCluster(CLUSTER_NAME, db, _numParitions + 10 * i, MASTER_SLAVE_STATE_MODEL,
+            .addResourceToCluster(CLUSTER_NAME, db, _numParitions + varyNum, MASTER_SLAVE_STATE_MODEL,
                 IdealState.RebalanceMode.FULL_AUTO.toString());
         _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _numReplicas);
         _testDbs.add(db);

http://git-wip-us.apache.org/repos/asf/helix/blob/ddc11f9a/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
new file mode 100644
index 0000000..1ced40b
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
@@ -0,0 +1,88 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Sets;
+
+public class TestWorkflowJobDependency extends TaskTestBase {
+  private static final Logger LOG = Logger.getLogger(TestWorkflowJobDependency.class);
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numDbs = 5;
+    _numParitions = 1;
+    _partitionVary = false;
+    super.beforeClass();
+  }
+
+  @Test (enabled = false)
+  public void testWorkflowWithOutDependencies() throws InterruptedException {
+    String workflowName = TestHelper.getTestMethodName();
+
+    // Workflow setup
+    LOG.info("Start setup for workflow: " + workflowName);
+    Workflow.Builder builder = new Workflow.Builder(workflowName);
+
+    for (int i = 0; i < _numDbs; i++) {
+      // Let each job delay for 2 secs.
+      JobConfig.Builder jobConfig = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+          .setTargetResource(_testDbs.get(i)).setTargetPartitionStates(Sets.newHashSet("SLAVE","MASTER"))
+          .setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG);
+      String jobName = "job" + _testDbs.get(i);
+      builder.addJob(jobName, jobConfig);
+    }
+
+    // Start workflow
+    Workflow workflow = builder.build();
+    _driver.start(workflow);
+
+    // Wait until the workflow completes
+    TaskTestUtil.pollForWorkflowState(_driver, workflowName, TaskState.COMPLETED);
+    WorkflowContext workflowContext = _driver.getWorkflowContext(workflowName);
+    long startTime = workflowContext.getStartTime();
+    long finishTime = workflowContext.getFinishTime();
+
+    // Update the start time range.
+    for (String jobName : workflow.getJobConfigs().keySet()) {
+      JobContext context = _driver.getJobContext(jobName);
+      LOG.info(String
+          .format("JOB: %s starts from %s finishes at %s.", jobName, context.getStartTime(),
+              context.getFinishTime()));
+
+      // Find job start time range.
+      startTime = Math.max(context.getStartTime(), startTime);
+      finishTime = Math.min(context.getFinishTime(), finishTime);
+    }
+
+    // All jobs have a valid overlap time range.
+    Assert.assertTrue(startTime <= finishTime);
+  }
+}


[22/33] helix git commit: Add unit test to retrieve workflow and job configurations.

Posted by lx...@apache.org.
Add unit test to retrieve workflow and job configurations.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/4f7fe130
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/4f7fe130
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/4f7fe130

Branch: refs/heads/helix-0.6.x
Commit: 4f7fe1306431210ee681a9eae64687f788f46bb3
Parents: 7f18483
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Apr 19 13:40:29 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:17:03 2016 -0700

----------------------------------------------------------------------
 .../integration/task/TestRetrieveWorkflows.java | 75 ++++++++++++++++++++
 1 file changed, 75 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/4f7fe130/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
new file mode 100644
index 0000000..786be7c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
@@ -0,0 +1,75 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.TestHelper;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowConfig;
+import org.apache.helix.task.WorkflowContext;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class TestRetrieveWorkflows extends TaskTestBase {
+  @Test
+  public void testGetAllWorkflows() throws Exception {
+    List<Workflow> workflowList = new ArrayList<Workflow>();
+    for (int i = 0; i < 4; i++) {
+      Workflow workflow = WorkflowGenerator.generateDefaultRepeatedJobWorkflowBuilder(TestHelper.getTestMethodName() + i).build();
+      _driver.start(workflow);
+      workflowList.add(workflow);
+    }
+
+    for (Workflow workflow : workflowList) {
+      //TaskTestUtil.pollForWorkflowState(_driver, workflow.getName(), TaskState.COMPLETED);
+    }
+
+    _manager = HelixManagerFactory
+        .getZKHelixManager("ESPRESSO_TEST_NUAGE", "Admin", InstanceType.ADMINISTRATOR, "zk-ei1-espresso.stg.linkedin.com:12913");
+    _manager.connect();
+    TaskDriver taskDriver = new TaskDriver(_manager);
+
+    Map<String, WorkflowConfig> workflowConfigMap = taskDriver.getWorkflows();
+    Assert.assertEquals(workflowConfigMap.size(), workflowList.size());
+
+    for(Map.Entry<String, WorkflowConfig> workflow :  workflowConfigMap.entrySet()) {
+      WorkflowConfig workflowConfig = workflow.getValue();
+      WorkflowContext workflowContext = _driver.getWorkflowContext(workflow.getKey());
+      Assert.assertNotNull(workflowContext);
+
+      for(String job : workflowConfig.getJobDag().getAllNodes()) {
+        JobConfig jobConfig = _driver.getJobConfig(job);
+        JobContext jobContext = _driver.getJobContext(job);
+
+        Assert.assertNotNull(jobConfig);
+        Assert.assertNotNull(jobContext);
+      }
+    }
+  }
+}
+


[33/33] helix git commit: Allow an instance group tag to be configured for a job, so all tasks of the job can only be running on the instances containing the tag.

Posted by lx...@apache.org.
Allow an instance group tag to be configured for a job, so all tasks of the job can only be running on the instances containing the tag.

1. Add instance group tag for jobs.
2. Add a test for job assignment when the only instance can be assigned instance is disabled.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/947a7d55
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/947a7d55
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/947a7d55

Branch: refs/heads/helix-0.6.x
Commit: 947a7d55756bdb4f50a3b0b358fb0364542df723
Parents: 6455b8b
Author: Junkai Xue <jx...@linkedin.com>
Authored: Mon Jun 6 13:19:53 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Wed Jul 6 13:18:53 2016 -0700

----------------------------------------------------------------------
 .../controller/stages/ClusterDataCache.java     | 23 ++++++++++++
 .../task/GenericTaskAssignmentCalculator.java   |  4 ++
 .../java/org/apache/helix/task/JobConfig.java   | 39 +++++++++++++++++---
 .../org/apache/helix/task/JobRebalancer.java    | 14 +++++--
 .../apache/helix/task/WorkflowRebalancer.java   |  4 ++
 .../org/apache/helix/task/beans/JobBean.java    |  1 +
 .../integration/task/TestTaskAssignment.java    | 37 +++++++++++++++++++
 7 files changed, 113 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
index fde4959..b77ce0d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
@@ -41,6 +41,7 @@ import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.task.JobConfig;
 import org.apache.log4j.Logger;
 
 import com.google.common.collect.Lists;
@@ -430,6 +431,28 @@ public class ClusterDataCache {
   }
 
   /**
+   * Return all the nodes that are enabled and tagged same as the job.
+   * @param allInstances List of instances to filter with instance tag
+   * @param instanceTag The instance group tag
+   * @return A new set contains instance name and that are marked enabled and have same
+   *         tag with job. The original set will not be changed during the filtering
+   */
+  public Set<String> getAllEnabledInstanceWithTag(final Set<String> allInstances,
+      String instanceTag) {
+    Set<String> enabledTagInstances = new HashSet<String>();
+    for (String instance : allInstances) {
+      InstanceConfig instanceConfig = _instanceConfigMap.get(instance);
+
+      if (instanceConfig != null && instanceConfig.getInstanceEnabled() && instanceConfig
+          .containsTag(instanceTag)) {
+        enabledTagInstances.add(instance);
+      }
+    }
+
+    return enabledTagInstances;
+  }
+
+  /**
    * Indicate that a full read should be done on the next refresh
    */
   public synchronized void requireFullRefresh() {

http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/main/java/org/apache/helix/task/GenericTaskAssignmentCalculator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/GenericTaskAssignmentCalculator.java b/helix-core/src/main/java/org/apache/helix/task/GenericTaskAssignmentCalculator.java
index e8d5f5d..b0a1a33 100644
--- a/helix-core/src/main/java/org/apache/helix/task/GenericTaskAssignmentCalculator.java
+++ b/helix-core/src/main/java/org/apache/helix/task/GenericTaskAssignmentCalculator.java
@@ -34,8 +34,10 @@ import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
 import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.ResourceAssignment;
+import org.apache.log4j.Logger;
 
 import com.google.common.base.Function;
 import com.google.common.collect.BiMap;
@@ -49,6 +51,8 @@ import com.google.common.collect.Sets;
  * assignment to target partitions and states of another resource
  */
 public class GenericTaskAssignmentCalculator extends TaskAssignmentCalculator {
+  private static final Logger LOG = Logger.getLogger(GenericTaskAssignmentCalculator.class);
+
   /** Reassignment policy for this algorithm */
   private RetryPolicy _retryPolicy = new DefaultRetryReassigner();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index d26c83b..7a4e2d3 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -117,7 +117,12 @@ public class JobConfig {
     /**
      * The type of the job
      */
-    JobType
+    JobType,
+
+    /**
+     * The instance group that task assign to
+     */
+    InstanceGroupTag
   }
 
   //Default property values
@@ -133,6 +138,7 @@ public class JobConfig {
   private final String _workflow;
   private final String _targetResource;
   private final String _jobType;
+  private final String _instanceGroupTag;
   private final List<String> _targetPartitions;
   private final Set<String> _targetPartitionStates;
   private final String _command;
@@ -152,7 +158,7 @@ public class JobConfig {
       long timeoutPerTask, int numConcurrentTasksPerInstance, int maxAttemptsPerTask,
       int maxForcedReassignmentsPerTask, int failureThreshold, long retryDelay,
       boolean disableExternalView, boolean ignoreDependentJobFailure,
-      Map<String, TaskConfig> taskConfigMap, String jobType) {
+      Map<String, TaskConfig> taskConfigMap, String jobType, String instanceGroupTag) {
     _workflow = workflow;
     _targetResource = targetResource;
     _targetPartitions = targetPartitions;
@@ -173,6 +179,7 @@ public class JobConfig {
       _taskConfigMap = Collections.emptyMap();
     }
     _jobType = jobType;
+    _instanceGroupTag = instanceGroupTag;
   }
 
   public String getWorkflow() {
@@ -274,9 +281,12 @@ public class JobConfig {
         "" + _numConcurrentTasksPerInstance);
     cfgMap.put(JobConfigProperty.IgnoreDependentJobFailure.name(),
         Boolean.toString(_ignoreDependentJobFailure));
-   if (_jobType != null) {
-     cfgMap.put(JobConfigProperty.JobType.name(), _jobType);
-   }
+    if (_jobType != null) {
+      cfgMap.put(JobConfigProperty.JobType.name(), _jobType);
+    }
+    if (_instanceGroupTag != null) {
+      cfgMap.put(JobConfigProperty.InstanceGroupTag.name(), _instanceGroupTag);
+    }
     return cfgMap;
   }
 
@@ -284,6 +294,10 @@ public class JobConfig {
     return _jobType;
   }
 
+  public String getInstanceGroupTag() {
+    return _instanceGroupTag;
+  }
+
   public static JobConfig fromHelixProperty(HelixProperty property)
       throws IllegalArgumentException {
     Map<String, String> configs = property.getRecord().getSimpleFields();
@@ -297,6 +311,7 @@ public class JobConfig {
     private String _workflow;
     private String _targetResource;
     private String _jobType;
+    private String _instanceGroupTag;
     private List<String> _targetPartitions;
     private Set<String> _targetPartitionStates;
     private String _command;
@@ -317,7 +332,8 @@ public class JobConfig {
       return new JobConfig(_workflow, _targetResource, _targetPartitions, _targetPartitionStates,
           _command, _commandConfig, _timeoutPerTask, _numConcurrentTasksPerInstance,
           _maxAttemptsPerTask, _maxForcedReassignmentsPerTask, _failureThreshold, _retryDelay,
-          _disableExternalView, _ignoreDependentJobFailure, _taskConfigMap, _jobType);
+          _disableExternalView, _ignoreDependentJobFailure, _taskConfigMap, _jobType,
+          _instanceGroupTag);
     }
 
     /**
@@ -382,6 +398,9 @@ public class JobConfig {
       if (cfg.containsKey(JobConfigProperty.JobType.name())) {
         b.setJobType(cfg.get(JobConfigProperty.JobType.name()));
       }
+      if (cfg.containsKey(JobConfigProperty.InstanceGroupTag.name())) {
+        b.setInstanceGroupTag(cfg.get(JobConfigProperty.InstanceGroupTag.name()));
+      }
       return b;
     }
 
@@ -474,6 +493,11 @@ public class JobConfig {
       return this;
     }
 
+    public Builder setInstanceGroupTag(String instanceGroupTag) {
+      _instanceGroupTag = instanceGroupTag;
+      return this;
+    }
+
     private void validate() {
       if (_taskConfigMap.isEmpty() && _targetResource == null) {
         throw new IllegalArgumentException(
@@ -555,6 +579,9 @@ public class JobConfig {
       if (jobBean.jobType != null) {
         b.setJobType(jobBean.jobType);
       }
+      if (jobBean.instanceGroupTag != null) {
+        b.setInstanceGroupTag(jobBean.instanceGroupTag);
+      }
       return b;
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index fae7ac7..c181ba5 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -128,11 +128,19 @@ public class JobRebalancer extends TaskRebalancer {
     // is stored in zk.
     // Fetch the previous resource assignment from the property store. This is required because of
     // HELIX-230.
+    Set<String> liveInstances = jobCfg.getInstanceGroupTag() == null
+        ? clusterData.getLiveInstances().keySet()
+        : clusterData.getAllEnabledInstanceWithTag(clusterData.getLiveInstances().keySet(),
+            jobCfg.getInstanceGroupTag());
+
+    if (liveInstances.isEmpty()) {
+      LOG.error("No available instance found for job!");
+    }
+
     Set<Integer> partitionsToDrop = new TreeSet<Integer>();
     ResourceAssignment newAssignment =
-        computeResourceMapping(jobName, workflowCfg, jobCfg, prevAssignment, clusterData
-            .getLiveInstances().keySet(), currStateOutput, workflowCtx, jobCtx, partitionsToDrop,
-            clusterData);
+        computeResourceMapping(jobName, workflowCfg, jobCfg, prevAssignment, liveInstances,
+            currStateOutput, workflowCtx, jobCtx, partitionsToDrop, clusterData);
 
     if (!partitionsToDrop.isEmpty()) {
       for (Integer pId : partitionsToDrop) {

http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
index 2d4ca75..b4f25d5 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
@@ -208,6 +208,10 @@ public class WorkflowRebalancer extends TaskRebalancer {
     builder.setNumPartitions(numPartitions);
     builder.setStateModel(TaskConstants.STATE_MODEL_NAME);
 
+    if (jobConfig.getInstanceGroupTag() != null) {
+      builder.setNodeGroup(jobConfig.getInstanceGroupTag());
+    }
+
     if (jobConfig.isDisableExternalView()) {
       builder.setDisableExternalView(true);
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java b/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
index 0080cc6..dd7ebab 100644
--- a/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
+++ b/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
@@ -32,6 +32,7 @@ public class JobBean {
   public List<String> parents;
   public String targetResource;
   public String jobType;
+  public String instanceGroupTag;
   public List<String> targetPartitionStates;
   public List<String> targetPartitions;
   public String command;

http://git-wip-us.apache.org/repos/asf/helix/blob/947a7d55/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
index df976b1..a22b63d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
@@ -19,8 +19,14 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.helix.TestHelper;
 import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.TaskConfig;
+import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.Workflow;
 import org.testng.Assert;
@@ -57,6 +63,37 @@ public class TestTaskAssignment extends TaskTestBase {
     // The task is not assigned so the task state should be null in this case.
     Assert.assertNull(
         _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource)).getPartitionState(0));
+  }
+
+  @Test
+  public void testGenericTaskInstanceGroup() throws InterruptedException {
+    // Disable the only instance can be assigned.
+    String queueName = TestHelper.getTestMethodName();
+    String jobName = "Job4InstanceGroup";
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
+    JobConfig.Builder jobConfig = new JobConfig.Builder();
+
+    List<TaskConfig> taskConfigs = new ArrayList<TaskConfig>();
+    int num_tasks = 3;
+    for (int j = 0; j < num_tasks; j++) {
+      taskConfigs.add(
+          new TaskConfig.Builder().setTaskId("task_" + j).setCommand(MockTask.TASK_COMMAND)
+              .build());
+    }
+
+    jobConfig.addTaskConfigs(taskConfigs);
+    jobConfig.setInstanceGroupTag("TESTTAG1");
+
+    queueBuilder.enqueueJob(jobName, jobConfig);
+    _driver.start(queueBuilder.build());
+
+    // Wait 1 sec. The task should not be complete since it is not assigned.
+    Thread.sleep(1000L);
+
+    // The task is not assigned so the task state should be null in this case.
+    String namedSpaceJob = TaskUtil.getNamespacedJobName(queueName, jobName);
 
+    Assert.assertEquals(_driver.getJobContext(namedSpaceJob).getAssignedParticipant(0),
+        _participants[1].getInstanceName());
   }
 }


[05/33] helix git commit: Add methods in TaskDriver for getting Workflow/Job configuration and context. External users should call these methods instead of TaskUtil.

Posted by lx...@apache.org.
Add methods in TaskDriver for getting Workflow/Job configuration and context. External users should call these methods instead of TaskUtil.


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

Branch: refs/heads/helix-0.6.x
Commit: d213c1a7b3fc310d38749f9555e5660858c37c3b
Parents: 7a47070
Author: Lei Xia <lx...@linkedin.com>
Authored: Fri Feb 12 15:12:06 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:42:18 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/TaskDriver.java  | 20 ++++++++++++++++++--
 1 file changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d213c1a7/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index 193526f..ce7bbf0 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -605,8 +605,8 @@ public class TaskDriver {
 
     // Set the job configuration
     PropertyKey.Builder keyBuilder = _accessor.keyBuilder();
-    HelixProperty resourceConfig = new HelixProperty(jobName);
-    resourceConfig.getRecord().getSimpleFields().putAll(jobConfig.getResourceConfigMap());
+    ResourceConfig resourceConfig = new ResourceConfig(jobName);
+    resourceConfig.putSimpleConfigs(jobConfig.getResourceConfigMap());
     Map<String, TaskConfig> taskConfigMap = jobConfig.getTaskConfigMap();
     if (taskConfigMap != null) {
       for (TaskConfig taskConfig : taskConfigMap.values()) {
@@ -695,6 +695,22 @@ public class TaskDriver {
     }
   }
 
+  public WorkflowConfig getWorkflowConfig(String workflow) {
+    return TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, workflow);
+  }
+
+  public WorkflowContext getWorkflowContext(String workflow) {
+    return TaskUtil.getWorkflowContext(_propertyStore, workflow);
+  }
+
+  public JobConfig getJobConfig(String job) {
+    return TaskUtil.getJobCfg(_accessor, job);
+  }
+
+  public JobContext getJobContext(String job) {
+    return TaskUtil.getJobContext(_propertyStore, job);
+  }
+
   public void list(String resource) {
     WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, resource);
     if (wCfg == null) {


[25/33] helix git commit: Add test to test task retry with and without delay.

Posted by lx...@apache.org.
Add test to test task retry with and without delay.


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

Branch: refs/heads/helix-0.6.x
Commit: e8ecdf389daf5976b726e132d05b343986eae5b2
Parents: ddefebb
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Apr 27 16:22:43 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:17:53 2016 -0700

----------------------------------------------------------------------
 .../apache/helix/integration/task/MockTask.java | 10 ++-
 .../integration/task/TestTaskRetryDelay.java    | 82 ++++++++++++++++++++
 2 files changed, 89 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/e8ecdf38/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
index f415b8e..3fe1d6f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
@@ -19,8 +19,6 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -34,10 +32,12 @@ public class MockTask implements Task {
   public static final String TIMEOUT_CONFIG = "Timeout";
   public static final String TASK_RESULT_STATUS = "TaskResultStatus";
   public static final String THROW_EXCEPTION = "ThrowException";
+  public static final String FAILURE_COUNT_BEFORE_SUCCESS = "FailureCountBeforeSuccess";
   private final long _delay;
   private volatile boolean _canceled;
   private TaskResult.Status _taskResultStatus;
   private boolean _throwException;
+  private int _expectedToSuccess;
 
   public MockTask(TaskCallbackContext context) {
     Map<String, String> cfg = context.getJobConfig().getJobCommandConfigMap();
@@ -58,6 +58,9 @@ public class MockTask implements Task {
     _throwException = cfg.containsKey(THROW_EXCEPTION) ?
         Boolean.valueOf(cfg.containsKey(THROW_EXCEPTION)) :
         false;
+    _expectedToSuccess =
+        cfg.containsKey(FAILURE_COUNT_BEFORE_SUCCESS) ? Integer.parseInt(cfg.get(
+            FAILURE_COUNT_BEFORE_SUCCESS)) : 0;
   }
 
   @Override
@@ -74,7 +77,8 @@ public class MockTask implements Task {
     }
     timeLeft = expiry - System.currentTimeMillis();
 
-    if (_throwException) {
+    if (_throwException || _expectedToSuccess > 0) {
+      _expectedToSuccess--;
       throw new RuntimeException("Test failed");
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/e8ecdf38/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
new file mode 100644
index 0000000..7d8ebff
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
@@ -0,0 +1,82 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class TestTaskRetryDelay extends TaskTestBase {
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numParitions = 1;
+    super.beforeClass();
+  }
+
+  @Test public void testTaskRetryWithDelay() throws Exception {
+    String jobResource = TestHelper.getTestMethodName();
+    JobConfig.Builder jobBuilder = JobConfig.Builder.fromMap(WorkflowGenerator.DEFAULT_JOB_CONFIG);
+    jobBuilder.setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG)
+        .setMaxAttemptsPerTask(2).setCommand(MockTask.TASK_COMMAND)
+        .setFailureThreshold(Integer.MAX_VALUE).setTaskRetryDelay(2000L)
+        .setJobCommandConfigMap(ImmutableMap.of(MockTask.FAILURE_COUNT_BEFORE_SUCCESS, "2"));
+    Workflow flow =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
+    _driver.start(flow);
+
+    // Wait until the job completes.
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+
+    long startTime = _driver.getWorkflowContext(jobResource).getStartTime();
+    long finishedTime = _driver.getWorkflowContext(jobResource).getFinishTime();
+
+    // It should finished at least 4 secs
+    Assert.assertTrue(finishedTime - startTime >= 2000L);
+  }
+
+  @Test public void testTaskRetryWithoutDelay() throws Exception {
+    String jobResource = TestHelper.getTestMethodName();
+    JobConfig.Builder jobBuilder = JobConfig.Builder.fromMap(WorkflowGenerator.DEFAULT_JOB_CONFIG);
+    jobBuilder.setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG)
+        .setMaxAttemptsPerTask(2).setCommand(MockTask.TASK_COMMAND)
+        .setFailureThreshold(Integer.MAX_VALUE)
+        .setJobCommandConfigMap(ImmutableMap.of(MockTask.FAILURE_COUNT_BEFORE_SUCCESS, "1"));
+    Workflow flow =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
+    _driver.start(flow);
+
+    // Wait until the job completes.
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+
+    long startTime = _driver.getWorkflowContext(jobResource).getStartTime();
+    long finishedTime = _driver.getWorkflowContext(jobResource).getFinishTime();
+
+    // It should finished at less than 2 sec
+    Assert.assertTrue(finishedTime - startTime <= 2000L);
+  }
+}
+


[27/33] helix git commit: Fix missing workflowtype assign in builder

Posted by lx...@apache.org.
Fix missing workflowtype assign in builder


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

Branch: refs/heads/helix-0.6.x
Commit: 7e9041f0d83a920f585300f8082bf4233d10f983
Parents: 2f62cd1
Author: Junkai Xue <jx...@linkedin.com>
Authored: Mon May 2 11:11:11 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:18:14 2016 -0700

----------------------------------------------------------------------
 helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/7e9041f0/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index 2881b61..a4356d2 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -269,6 +269,7 @@ public class  WorkflowConfig {
       _scheduleConfig = workflowConfig.getScheduleConfig();
       _capacity = workflowConfig.getCapacity();
       _failureThreshold = workflowConfig.getFailureThreshold();
+      _workflowType =workflowConfig.getWorkflowType();
     }
 
     protected Builder setJobDag(JobDag v) {


[29/33] helix git commit: Populate Error message from running client's task and persist it into JobContext for better error reporting.

Posted by lx...@apache.org.
Populate Error message from running client's task and persist it into JobContext for better error reporting.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/9508a1ac
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/9508a1ac
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/9508a1ac

Branch: refs/heads/helix-0.6.x
Commit: 9508a1acfae1d915148138daccb2abd5f9dce430
Parents: c3624e0
Author: Lei Xia <lx...@linkedin.com>
Authored: Thu May 5 11:25:22 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:18:34 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobContext.java  |  51 ++++++--
 .../org/apache/helix/task/JobRebalancer.java    |  15 ++-
 .../java/org/apache/helix/task/TaskRunner.java  |   5 +-
 .../org/apache/helix/task/TaskStateModel.java   |   4 +-
 .../apache/helix/integration/task/MockTask.java |  23 +++-
 .../task/TestTaskErrorReporting.java            | 117 +++++++++++++++++++
 6 files changed, 194 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9508a1ac/helix-core/src/main/java/org/apache/helix/task/JobContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobContext.java b/helix-core/src/main/java/org/apache/helix/task/JobContext.java
index 2057f27..328fcc0 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobContext.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobContext.java
@@ -45,7 +45,8 @@ public class JobContext extends HelixProperty {
     TARGET,
     TASK_ID,
     ASSIGNED_PARTICIPANT,
-    NEXT_RETRY_TIME
+    NEXT_RETRY_TIME,
+    INFO
   }
 
   public JobContext(ZNRecord record) {
@@ -76,8 +77,18 @@ public class JobContext extends HelixProperty {
     return Long.parseLong(tStr);
   }
 
+  public void setInfo(String info) {
+    if (info != null) {
+      _record.setSimpleField(ContextProperties.INFO.toString(), info);
+    }
+  }
+
+  public String getInfo() {
+    return _record.getSimpleField(ContextProperties.INFO.toString());
+  }
+
   public void setPartitionState(int p, TaskPartitionState s) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.STATE.toString(), s.name());
   }
 
@@ -95,7 +106,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setPartitionNumAttempts(int p, int n) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.NUM_ATTEMPTS.toString(), String.valueOf(n));
   }
 
@@ -122,7 +133,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setPartitionStartTime(int p, long t) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.START_TIME.toString(), String.valueOf(t));
   }
 
@@ -139,7 +150,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setPartitionFinishTime(int p, long t) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.FINISH_TIME.toString(), String.valueOf(t));
   }
 
@@ -156,7 +167,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setPartitionTarget(int p, String targetPName) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.TARGET.toString(), targetPName);
   }
 
@@ -165,6 +176,16 @@ public class JobContext extends HelixProperty {
     return (map != null) ? map.get(ContextProperties.TARGET.toString()) : null;
   }
 
+  public void setPartitionInfo(int p, String info) {
+    Map<String, String> map = getMapField(p, true);
+    map.put(ContextProperties.INFO.toString(), info);
+  }
+
+  public String getPartitionInfo(int p) {
+    Map<String, String> map = getMapField(p);
+    return (map != null) ? map.get(ContextProperties.INFO.toString()) : null;
+  }
+
   public Map<String, List<Integer>> getPartitionsByTarget() {
     Map<String, List<Integer>> result = Maps.newHashMap();
     for (Map.Entry<String, Map<String, String>> mapField : _record.getMapFields().entrySet()) {
@@ -194,7 +215,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setTaskIdForPartition(int p, String taskId) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.TASK_ID.toString(), taskId);
   }
 
@@ -216,7 +237,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setAssignedParticipant(int p, String participantName) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.ASSIGNED_PARTICIPANT.toString(), participantName);
   }
 
@@ -226,7 +247,7 @@ public class JobContext extends HelixProperty {
   }
 
   public void setNextRetryTime(int p, long t) {
-    Map<String, String> map = getMapField(p);
+    Map<String, String> map = getMapField(p, true);
     map.put(ContextProperties.NEXT_RETRY_TIME.toString(), String.valueOf(t));
   }
 
@@ -242,10 +263,20 @@ public class JobContext extends HelixProperty {
     return Long.parseLong(tStr);
   }
 
+  /**
+   * Get MapField for the given partition.
+   *
+   * @param p
+   * @return mapField for the partition, NULL if the partition has not scheduled yet.
+   */
   public Map<String, String> getMapField(int p) {
+    return getMapField(p, false);
+  }
+
+  private Map<String, String> getMapField(int p, boolean createIfNotPresent) {
     String pStr = String.valueOf(p);
     Map<String, String> map = _record.getMapField(pStr);
-    if (map == null) {
+    if (map == null && createIfNotPresent) {
       map = new TreeMap<String, String>();
       _record.setMapField(pStr, map);
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/9508a1ac/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index 0f34178..fae7ac7 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -214,8 +214,9 @@ public class JobRebalancer extends TaskRebalancer {
 
     if (allPartitions == null || allPartitions.isEmpty()) {
       // Empty target partitions, mark the job as FAILED.
-      LOG.warn(
-          "Missing task partition mapping for job " + jobResource + ", marked the job as FAILED!");
+      String failureMsg = "Empty task partition mapping for job " + jobResource + ", marked the job as FAILED!";
+      LOG.info(failureMsg);
+      jobCtx.setInfo(failureMsg);
       markJobFailed(jobResource, jobCtx, workflowConfig, workflowCtx);
       markAllPartitionsError(jobCtx, TaskPartitionState.ERROR, false);
       return new ResourceAssignment(jobResource);
@@ -266,6 +267,12 @@ public class JobRebalancer extends TaskRebalancer {
                 pName), instance));
         jobCtx.setPartitionState(pId, currState);
 
+        String taskMsg = currStateOutput.getInfo(jobResource, new Partition(
+            pName), instance);
+        if (taskMsg != null) {
+          jobCtx.setPartitionInfo(pId, taskMsg);
+        }
+
         // Process any requested state transitions.
         String requestedStateStr =
             currStateOutput.getRequestedState(jobResource, new Partition(pName), instance);
@@ -318,8 +325,8 @@ public class JobRebalancer extends TaskRebalancer {
         case ERROR: {
           donePartitions.add(pId); // The task may be rescheduled on a different instance.
           LOG.debug(String.format(
-              "Task partition %s has error state %s. Marking as such in rebalancer context.", pName,
-              currState));
+              "Task partition %s has error state %s with msg %s. Marking as such in rebalancer context.", pName,
+              currState, taskMsg));
           markPartitionError(jobCtx, pId, currState, true);
           // The error policy is to fail the task as soon a single partition fails for a specified
           // maximum number of attempts or task is in ABORTED state.

http://git-wip-us.apache.org/repos/asf/helix/blob/9508a1ac/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java b/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
index c43d0ce..eabaf64 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
@@ -72,7 +72,7 @@ public class TaskRunner implements Runnable {
         throw death;
       } catch (Throwable t) {
         LOG.error("Problem running the task, report task as FAILED.", t);
-        _result = new TaskResult(Status.FAILED, null);
+        _result = new TaskResult(Status.FAILED, "Exception happened in running task: " + t.getMessage());
       }
 
       switch (_result.getStatus()) {
@@ -98,6 +98,9 @@ public class TaskRunner implements Runnable {
         throw new AssertionError("Unknown task result type: " + _result.getStatus().name());
       }
     } catch (Exception e) {
+      LOG.error("Problem running the task, report task as FAILED.", e);
+      _result =
+          new TaskResult(Status.FAILED, "Exception happened in running task: " + e.getMessage());
       requestStateTransition(TaskPartitionState.TASK_ERROR);
     } finally {
       synchronized (_doneSync) {

http://git-wip-us.apache.org/repos/asf/helix/blob/9508a1ac/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java b/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
index ba68a78..fd07176 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
@@ -87,7 +87,7 @@ public class TaskStateModel extends StateModel {
   }
 
   @Transition(to = "COMPLETED", from = "RUNNING")
-  public void onBecomeCompletedFromRunning(Message msg, NotificationContext context) {
+  public String onBecomeCompletedFromRunning(Message msg, NotificationContext context) {
     String taskPartition = msg.getPartitionName();
     if (_taskRunner == null) {
       throw new IllegalStateException(String.format(
@@ -102,6 +102,8 @@ public class TaskStateModel extends StateModel {
     }
 
     timeout_task.cancel(false);
+
+    return r.getInfo();
   }
 
   @Transition(to = "TIMED_OUT", from = "RUNNING")

http://git-wip-us.apache.org/repos/asf/helix/blob/9508a1ac/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
index 3fe1d6f..db0c8f4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
@@ -32,12 +32,14 @@ public class MockTask implements Task {
   public static final String TIMEOUT_CONFIG = "Timeout";
   public static final String TASK_RESULT_STATUS = "TaskResultStatus";
   public static final String THROW_EXCEPTION = "ThrowException";
+  public static final String ERROR_MESSAGE = "ErrorMessage";
   public static final String FAILURE_COUNT_BEFORE_SUCCESS = "FailureCountBeforeSuccess";
   private final long _delay;
   private volatile boolean _canceled;
   private TaskResult.Status _taskResultStatus;
   private boolean _throwException;
   private int _expectedToSuccess;
+  private String _errorMsg;
 
   public MockTask(TaskCallbackContext context) {
     Map<String, String> cfg = context.getJobConfig().getJobCommandConfigMap();
@@ -59,8 +61,10 @@ public class MockTask implements Task {
         Boolean.valueOf(cfg.containsKey(THROW_EXCEPTION)) :
         false;
     _expectedToSuccess =
-        cfg.containsKey(FAILURE_COUNT_BEFORE_SUCCESS) ? Integer.parseInt(cfg.get(
-            FAILURE_COUNT_BEFORE_SUCCESS)) : 0;
+        cfg.containsKey(FAILURE_COUNT_BEFORE_SUCCESS) ? Integer.parseInt(
+            cfg.get(FAILURE_COUNT_BEFORE_SUCCESS)) : 0;
+
+    _errorMsg = cfg.containsKey(ERROR_MESSAGE) ? cfg.get(ERROR_MESSAGE) : null;
   }
 
   @Override
@@ -77,12 +81,21 @@ public class MockTask implements Task {
     }
     timeLeft = expiry - System.currentTimeMillis();
 
-    if (_throwException || _expectedToSuccess > 0) {
+    if (_throwException) {
+      _expectedToSuccess--;
+      if (_errorMsg == null) {
+        _errorMsg = "Test failed";
+      }
+      throw new RuntimeException(_errorMsg != null ? _errorMsg : "Test failed");
+    }
+
+    if (_expectedToSuccess > 0){
       _expectedToSuccess--;
-      throw new RuntimeException("Test failed");
+      throw new RuntimeException(_errorMsg != null ? _errorMsg : "Test failed");
     }
 
-    return new TaskResult(_taskResultStatus, String.valueOf(timeLeft < 0 ? 0 : timeLeft));
+    return new TaskResult(_taskResultStatus,
+        _errorMsg != null ? _errorMsg : String.valueOf(timeLeft < 0 ? 0 : timeLeft));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/9508a1ac/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
new file mode 100644
index 0000000..906dcff
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
@@ -0,0 +1,117 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.TaskConfig;
+import org.apache.helix.task.TaskPartitionState;
+import org.apache.helix.task.TaskResult;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test Error reporting for failed tasks
+ */
+public class TestTaskErrorReporting extends TaskTestBase {
+
+  @Test
+  public void test() throws Exception {
+    int taskRetryCount = 1;
+    int num_tasks = 5;
+
+    String jobResource = TestHelper.getTestMethodName();
+    JobConfig.Builder jobBuilder = new JobConfig.Builder();
+    jobBuilder.setCommand(MockTask.TASK_COMMAND).setTimeoutPerTask(10000)
+        .setMaxAttemptsPerTask(taskRetryCount).setFailureThreshold(Integer.MAX_VALUE);
+
+    // create each task configs.
+    final int abortedTask = 1;
+    final int failedTask = 2;
+    final int exceptionTask = 3;
+
+    final String abortedMsg = "This task aborted, some terrible things must happened.";
+    final String failedMsg = "This task failed, something may be wrong.";
+    final String exceptionMsg = "This task throws exception.";
+    final String successMsg = "Yes, we did it!";
+
+    List<TaskConfig> taskConfigs = new ArrayList<TaskConfig>();
+    for (int j = 0; j < num_tasks; j++) {
+      TaskConfig.Builder configBuilder = new TaskConfig.Builder().setTaskId("task_" + j);
+      switch (j) {
+      case abortedTask:
+        configBuilder.addConfig(MockTask.TASK_RESULT_STATUS, TaskResult.Status.FATAL_FAILED.name())
+            .addConfig(MockTask.ERROR_MESSAGE, abortedMsg);
+        break;
+      case failedTask:
+        configBuilder.addConfig(MockTask.TASK_RESULT_STATUS, TaskResult.Status.FAILED.name())
+            .addConfig(MockTask.ERROR_MESSAGE, failedMsg);
+        break;
+      case exceptionTask:
+        configBuilder.addConfig(MockTask.THROW_EXCEPTION, Boolean.TRUE.toString())
+            .addConfig(MockTask.ERROR_MESSAGE, exceptionMsg);
+        break;
+      default:
+        configBuilder.addConfig(MockTask.ERROR_MESSAGE, successMsg);
+        break;
+      }
+      configBuilder.setTargetPartition(String.valueOf(j));
+      taskConfigs.add(configBuilder.build());
+    }
+    jobBuilder.addTaskConfigs(taskConfigs);
+
+    Workflow flow =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
+
+    _driver.start(flow);
+
+    // Wait until the job completes.
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+
+    JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
+    for (int i = 0; i < num_tasks; i++) {
+      TaskPartitionState state = ctx.getPartitionState(i);
+      String taskId = ctx.getTaskIdForPartition(i);
+      String errMsg = ctx.getPartitionInfo(i);
+
+      if (taskId.equals("task_" + abortedTask)) {
+        Assert.assertEquals(state, TaskPartitionState.TASK_ABORTED);
+        Assert.assertEquals(errMsg, abortedMsg);
+      } else if (taskId.equals("task_" + failedTask)) {
+        Assert.assertEquals(state, TaskPartitionState.TASK_ERROR);
+        Assert.assertEquals(errMsg, failedMsg);
+      } else if (taskId.equals("task_" + exceptionTask)) {
+        Assert.assertEquals(state, TaskPartitionState.TASK_ERROR);
+        Assert.assertTrue(errMsg.contains(exceptionMsg));
+      } else {
+        Assert.assertEquals(state, TaskPartitionState.COMPLETED);
+        Assert.assertEquals(errMsg, successMsg);
+
+      }
+    }
+  }
+}


[04/33] helix git commit: Support changes of workflow configuration.

Posted by lx...@apache.org.
Support changes of workflow configuration.


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

Branch: refs/heads/helix-0.6.x
Commit: 7a47070362c536126e945138f10fb077215f706e
Parents: 3781929
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Jan 26 09:57:43 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:34:59 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/HelixException.java   |   4 +
 .../main/java/org/apache/helix/task/JobDag.java |   4 +-
 .../java/org/apache/helix/task/TaskDriver.java  | 133 ++++++++---
 .../org/apache/helix/task/WorkflowConfig.java   |  24 +-
 .../helix/integration/task/TaskTestUtil.java    |  36 +--
 .../task/TestRunJobsWithMissingTarget.java      |   9 +-
 .../integration/task/TestUpdateWorkflow.java    | 220 +++++++++++++++++++
 7 files changed, 382 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/main/java/org/apache/helix/HelixException.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixException.java b/helix-core/src/main/java/org/apache/helix/HelixException.java
index 8693026..26585ed 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixException.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixException.java
@@ -33,4 +33,8 @@ public class HelixException extends RuntimeException {
   public HelixException(Throwable cause) {
     super(cause);
   }
+
+  public HelixException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/main/java/org/apache/helix/task/JobDag.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobDag.java b/helix-core/src/main/java/org/apache/helix/task/JobDag.java
index f708e91..73a5e58 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobDag.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobDag.java
@@ -19,12 +19,14 @@ package org.apache.helix.task;
  * under the License.
  */
 
+import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -141,7 +143,7 @@ public class JobDag {
     return ret;
   }
 
-  public String toJson() throws Exception {
+  public String toJson() throws IOException {
     return new ObjectMapper().writeValueAsString(this);
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index c4986ee..193526f 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -42,6 +42,7 @@ import org.apache.helix.AccessOption;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.HelixProperty;
@@ -183,8 +184,11 @@ public class TaskDriver {
     helixMgr.disconnect();
   }
 
-  /** Schedules a new workflow */
-  public void start(Workflow flow) throws Exception {
+  /** Schedules a new workflow
+   *
+   * @param flow
+   */
+  public void start(Workflow flow) {
     // TODO: check that namespace for workflow is available
     LOG.info("Starting workflow " + flow.getName());
     flow.validate();
@@ -206,14 +210,65 @@ public class TaskDriver {
     addWorkflowResource(flow.getName());
   }
 
-  /** Creates a new named job queue (workflow) */
-  public void createQueue(JobQueue queue) throws Exception {
+  /**
+   * Update the configuration of a non-terminable workflow (queue).
+   * The terminable workflow's configuration is not allowed
+   * to change once created.
+   * Note:
+   * For recurrent workflow, the current running schedule will not be effected,
+   * the new configuration will be applied to the next scheduled runs of the workflow.
+   * For non-recurrent workflow, the new configuration may (or may not) be applied
+   * on the current running jobs, but it will be applied on the following unscheduled jobs.
+   *
+   * Example:
+   *
+   * WorkflowConfig currentWorkflowConfig = TaskUtil.getWorkflowCfg(_manager, workflow);
+   * WorkflowConfig.Builder configBuilder = new WorkflowConfig.Builder(currentWorkflowConfig);
+
+   * // make needed changes to the config here
+   * configBuilder.setXXX();
+   *
+   * // update workflow configuration
+   * _driver.updateWorkflow(workflow, configBuilder.build());
+   *
+   * @param workflow
+   * @param newWorkflowConfig
+   */
+  public void updateWorkflow(String workflow, WorkflowConfig newWorkflowConfig) {
+    WorkflowConfig currentConfig =
+        TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, workflow);
+    if (currentConfig == null) {
+      throw new HelixException("Workflow " + workflow + " does not exist!");
+    }
+
+    if (currentConfig.isTerminable()) {
+      throw new HelixException(
+          "Workflow " + workflow + " is terminable, not allow to change its configuration!");
+    }
+
+    _admin.setConfig(TaskUtil.getResourceConfigScope(_clusterName, workflow),
+        newWorkflowConfig.getResourceConfigMap());
+
+    TaskUtil.invokeRebalance(_accessor, workflow);
+  }
+
+  /**
+   * Creates a new named job queue (workflow)
+   *
+   * @param queue
+   */
+  public void createQueue(JobQueue queue) {
     start(queue);
   }
 
-  /** Flushes a named job queue */
+  /**
+   * Flushes a named job queue
+   *
+   * @param queueName
+   * @throws Exception
+   */
   // TODO: need to make sure the queue is stopped or completed before flush the queue.
-  public void flushQueue(String queueName) throws Exception {
+  public void flushQueue(String queueName) {
     WorkflowConfig config =
         TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, queueName);
     if (config == null) {
@@ -275,7 +330,13 @@ public class TaskDriver {
     _propertyStore.update(path, updater, AccessOption.PERSISTENT);
   }
 
-  /** Delete a job from an existing named queue, the queue has to be stopped prior to this call */
+  /**
+   * Delete a job from an existing named queue,
+   * the queue has to be stopped prior to this call
+   *
+   * @param queueName
+   * @param jobName
+   */
   public void deleteJob(final String queueName, final String jobName) {
     WorkflowConfig workflowCfg =
         TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, queueName);
@@ -315,8 +376,12 @@ public class TaskDriver {
     }
   }
 
-
-  /** delete a job from a scheduled (non-recurrent) queue.*/
+  /**
+   * delete a job from a scheduled (non-recurrent) queue.
+   *
+   * @param queueName
+   * @param jobName
+   */
   private void deleteJobFromScheduledQueue(final String queueName, final String jobName) {
     WorkflowConfig workflowCfg =
         TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, queueName);
@@ -354,9 +419,7 @@ public class TaskDriver {
     _propertyStore.remove(jobPropertyPath, AccessOption.PERSISTENT);
   }
 
-  /**
-   * Remove the job name from the DAG from the queue configuration
-   */
+  /** Remove the job name from the DAG from the queue configuration */
   private void removeJobFromDag(final String queueName, final String jobName) {
     final String namespacedJobName = TaskUtil.getNamespacedJobName(queueName, jobName);
 
@@ -410,8 +473,7 @@ public class TaskDriver {
     }
   }
 
-  /** update queue's property to remove job from JOB_STATES if it is already started.
-   */
+  /** update queue's property to remove job from JOB_STATES if it is already started. */
   private void removeJobStateFromQueue(final String queueName, final String jobName) {
     final String namespacedJobName = TaskUtil.getNamespacedJobName(queueName, jobName);
     String queuePropertyPath =
@@ -435,9 +497,16 @@ public class TaskDriver {
     }
   }
 
-  /** Adds a new job to the end an existing named queue */
-  public void enqueueJob(final String queueName, final String jobName, JobConfig.Builder jobBuilder)
-      throws Exception {
+  /**
+   * Adds a new job to the end an existing named queue.
+   *
+   * @param queueName
+   * @param jobName
+   * @param jobBuilder
+   * @throws Exception
+   */
+  public void enqueueJob(final String queueName, final String jobName,
+      JobConfig.Builder jobBuilder) {
     // Get the job queue config and capacity
     HelixProperty workflowConfig =
         _accessor.getProperty(_accessor.keyBuilder().resourceConfig(queueName));
@@ -468,12 +537,12 @@ public class TaskDriver {
         JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
         Set<String> allNodes = jobDag.getAllNodes();
         if (allNodes.size() >= capacity) {
-          throw new IllegalStateException("Queue " + queueName + " is at capacity, will not add "
-              + jobName);
+          throw new IllegalStateException(
+              "Queue " + queueName + " is at capacity, will not add " + jobName);
         }
         if (allNodes.contains(namespacedJobName)) {
-          throw new IllegalStateException("Could not add to queue " + queueName + ", job "
-              + jobName + " already exists");
+          throw new IllegalStateException(
+              "Could not add to queue " + queueName + ", job " + jobName + " already exists");
         }
         jobDag.addNode(namespacedJobName);
 
@@ -493,8 +562,8 @@ public class TaskDriver {
         try {
           currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
         } catch (Exception e) {
-          throw new IllegalStateException(
-              "Could not add job " + jobName + " to queue " + queueName, e);
+          throw new IllegalStateException("Could not add job " + jobName + " to queue " + queueName,
+              e);
         }
         return currentData;
       }
@@ -550,17 +619,29 @@ public class TaskDriver {
     }
   }
 
-  /** Public method to resume a workflow/queue */
+  /**
+   * Public method to resume a workflow/queue.
+   *
+   * @param workflow
+   */
   public void resume(String workflow) {
     setWorkflowTargetState(workflow, TargetState.START);
   }
 
-  /** Public method to stop a workflow/queue */
+  /**
+   * Public method to stop a workflow/queue.
+   *
+   * @param workflow
+   */
   public void stop(String workflow) {
     setWorkflowTargetState(workflow, TargetState.STOP);
   }
 
-  /** Public method to delete a workflow/queue */
+  /**
+   * Public method to delete a workflow/queue.
+   *
+   * @param workflow
+   */
   public void delete(String workflow) {
     setWorkflowTargetState(workflow, TargetState.DELETE);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index 56fba58..4c81654 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -18,13 +18,14 @@ package org.apache.helix.task;
  * specific language governing permissions and limitations
  * under the License.
  */
-
+import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
+import org.apache.helix.HelixException;
 
 /**
  * Provides a typed interface to workflow level configurations. Validates the configurations.
@@ -44,7 +45,9 @@ public class WorkflowConfig {
   public static final long DEFAULT_EXPIRY = 24 * 60 * 60 * 1000;
 
   /* Member variables */
+  // TODO: jobDag should not be in the workflowConfig.
   private final JobDag _jobDag;
+
   // _parallelJobs would kind of break the job dependency,
   // e.g: if job1 -> job2, but _parallelJobs == 2,
   // then job1 and job2 could be scheduled at the same time
@@ -114,9 +117,13 @@ public class WorkflowConfig {
     return _scheduleConfig.getStartTime();
   }
 
-  public Map<String, String> getResourceConfigMap() throws Exception {
+  public Map<String, String> getResourceConfigMap() {
     Map<String, String> cfgMap = new HashMap<String, String>();
-    cfgMap.put(WorkflowConfig.DAG, getJobDag().toJson());
+    try {
+      cfgMap.put(WorkflowConfig.DAG, getJobDag().toJson());
+    } catch (IOException ex) {
+      throw new HelixException("Invalid job dag configuration!", ex);
+    }
     cfgMap.put(WorkflowConfig.PARALLEL_JOBS, String.valueOf(getParallelJobs()));
     cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(getExpiry()));
     cfgMap.put(WorkflowConfig.TARGET_STATE, getTargetState().name());
@@ -153,6 +160,17 @@ public class WorkflowConfig {
       return new WorkflowConfig(_taskDag, _parallelJobs, _targetState, _expiry, _isTerminable, _scheduleConfig);
     }
 
+    public Builder() {}
+
+    public Builder(WorkflowConfig workflowConfig) {
+      _taskDag = workflowConfig.getJobDag();
+      _parallelJobs = workflowConfig.getParallelJobs();
+      _targetState = workflowConfig.getTargetState();
+      _expiry = workflowConfig.getExpiry();
+      _isTerminable = workflowConfig.isTerminable();
+      _scheduleConfig = workflowConfig.getScheduleConfig();
+    }
+
     public Builder setJobDag(JobDag v) {
       _taskDag = v;
       return this;

http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index 11677b8..06b9751 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -49,22 +49,26 @@ public class TaskTestUtil {
    * Polls {@link org.apache.helix.task.JobContext} for given task resource until a timeout is
    * reached.
    * If the task has not reached target state by then, an error is thrown
+   *
    * @param workflowResource Resource to poll for completeness
    * @throws InterruptedException
    */
   public static void pollForWorkflowState(HelixManager manager, String workflowResource,
-      TaskState state) throws InterruptedException {
+      TaskState... targetStates) throws InterruptedException {
     // Wait for completion.
     long st = System.currentTimeMillis();
     WorkflowContext ctx;
+    Set<TaskState> allowedStates = new HashSet<TaskState>(Arrays.asList(targetStates));
     do {
       Thread.sleep(100);
       ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
-    } while ((ctx == null || ctx.getWorkflowState() == null || ctx.getWorkflowState() != state)
-        && System.currentTimeMillis() < st + _default_timeout);
+    } while ((ctx == null || ctx.getWorkflowState() == null || !allowedStates
+        .contains(ctx.getWorkflowState())) && System.currentTimeMillis() < st + _default_timeout);
 
     Assert.assertNotNull(ctx);
-    Assert.assertEquals(ctx.getWorkflowState(), state);
+    TaskState workflowState = ctx.getWorkflowState();
+    Assert.assertTrue(allowedStates.contains(workflowState),
+        "expect workflow states: " + allowedStates + " actual workflow state: " + workflowState);
   }
 
   /**
@@ -101,10 +105,13 @@ public class TaskTestUtil {
       Thread.sleep(100);
       ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
     }
-    while ((ctx == null || ctx.getJobState(jobName) == null || !allowedStates.contains(ctx.getJobState(jobName)))
+    while ((ctx == null || ctx.getJobState(jobName) == null || !allowedStates.contains(
+        ctx.getJobState(jobName)))
         && System.currentTimeMillis() < st + _default_timeout);
-    Assert.assertNotNull(ctx);
-    Assert.assertTrue(allowedStates.contains(ctx.getJobState(jobName)));
+    Assert.assertNotNull(ctx, "Empty job context");
+    TaskState jobState = ctx.getJobState(jobName);
+    Assert.assertTrue(allowedStates.contains(jobState),
+        "expect job states: " + allowedStates + " actual job state: " + jobState);
   }
 
   public static void pollForEmptyJobState(final HelixManager manager, final String workflowName,
@@ -127,8 +134,8 @@ public class TaskTestUtil {
     long st = System.currentTimeMillis();
     WorkflowContext ctx;
     do {
-      Thread.sleep(100);
       ctx = TaskUtil.getWorkflowContext(manager, workflowResource);
+      Thread.sleep(100);
     } while (ctx == null && System.currentTimeMillis() < st + _default_timeout);
     Assert.assertNotNull(ctx);
     return ctx;
@@ -228,9 +235,14 @@ public class TaskTestUtil {
   }
 
   public static JobQueue.Builder buildRecurrentJobQueue(String jobQueueName, int delayStart) {
+    return buildRecurrentJobQueue(jobQueueName, delayStart, 60);
+  }
+
+  public static JobQueue.Builder buildRecurrentJobQueue(String jobQueueName, int delayStart,
+      int recurrenInSeconds) {
     Map<String, String> cfgMap = new HashMap<String, String>();
     cfgMap.put(WorkflowConfig.EXPIRY, String.valueOf(120000));
-    cfgMap.put(WorkflowConfig.RECURRENCE_INTERVAL, String.valueOf(60));
+    cfgMap.put(WorkflowConfig.RECURRENCE_INTERVAL, String.valueOf(recurrenInSeconds));
     cfgMap.put(WorkflowConfig.RECURRENCE_UNIT, "SECONDS");
     Calendar cal = Calendar.getInstance();
     cal.set(Calendar.MINUTE, cal.get(Calendar.MINUTE) + delayStart / 60);
@@ -238,8 +250,6 @@ public class TaskTestUtil {
     cal.set(Calendar.MILLISECOND, 0);
     cfgMap.put(WorkflowConfig.START_TIME,
         WorkflowConfig.getDefaultDateFormat().format(cal.getTime()));
-    //cfgMap.put(WorkflowConfig.START_TIME,
-    //WorkflowConfig.getDefaultDateFormat().format(getDateFromStartTime("00:00")));
     return new JobQueue.Builder(jobQueueName).fromMap(cfgMap);
   }
 
@@ -262,8 +272,4 @@ public class TaskTestUtil {
   public static JobQueue.Builder buildJobQueue(String jobQueueName) {
     return buildJobQueue(jobQueueName, 0);
   }
-
-  public static boolean pollForParticipantParallelState() {
-    return false;
-  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index 31e4325..9fd7735 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -68,7 +68,8 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
   private HelixManager _manager;
   private TaskDriver _driver;
 
-  @BeforeClass public void beforeClass() throws Exception {
+  @BeforeClass
+  public void beforeClass() throws Exception {
     String namespace = "/" + CLUSTER_NAME;
     if (_gZkClient.exists(namespace)) {
       _gZkClient.deleteRecursive(namespace);
@@ -141,7 +142,8 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     }
   }
 
-  @Test public void testJobFailsWithMissingTarget() throws Exception {
+  @Test
+  public void testJobFailsWithMissingTarget() throws Exception {
     String queueName = TestHelper.getTestMethodName();
 
     // Create a queue
@@ -166,7 +168,8 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.FAILED);
   }
 
-  @Test public void testJobFailsWithMissingTargetInRunning() throws Exception {
+  @Test
+  public void testJobFailsWithMissingTargetInRunning() throws Exception {
     String queueName = TestHelper.getTestMethodName();
 
     // Create a queue

http://git-wip-us.apache.org/repos/asf/helix/blob/7a470703/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
new file mode 100644
index 0000000..fc93392
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
@@ -0,0 +1,220 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.ScheduleConfig;
+import org.apache.helix.task.Task;
+import org.apache.helix.task.TaskCallbackContext;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowConfig;
+import org.apache.helix.task.WorkflowContext;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class TestUpdateWorkflow extends ZkIntegrationTestBase {
+  private static final Logger LOG = Logger.getLogger(TestUpdateWorkflow.class);
+  private static final int n = 5;
+  private static final int START_PORT = 12918;
+  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
+  private static final String TIMEOUT_CONFIG = "Timeout";
+  private static final String TGT_DB = "TestDB";
+  private static final int NUM_PARTITIONS = 20;
+  private static final int NUM_REPLICAS = 3;
+  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
+  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
+  private ClusterControllerManager _controller;
+
+  private HelixManager _manager;
+  private TaskDriver _driver;
+  private ZKHelixDataAccessor _accessor;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+
+    _accessor =
+        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+
+    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
+    setupTool.addCluster(CLUSTER_NAME, true);
+    for (int i = 0; i < n; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    }
+
+    // Set up target db
+    setupTool.addResourceToCluster(CLUSTER_NAME, TGT_DB, NUM_PARTITIONS, MASTER_SLAVE_STATE_MODEL);
+    setupTool.rebalanceStorageCluster(CLUSTER_NAME, TGT_DB, NUM_REPLICAS);
+
+    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
+    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
+      @Override
+      public Task createNewTask(TaskCallbackContext context) {
+        return new MockTask(context);
+      }
+    });
+
+    // start dummy participants
+    for (int i = 0; i < n; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+
+      // Register a Task state model factory.
+      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
+      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
+          taskFactoryReg));
+
+      _participants[i].syncStart();
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // create cluster manager
+    _manager =
+        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
+            ZK_ADDR);
+    _manager.connect();
+
+    _driver = new TaskDriver(_manager);
+
+    boolean result =
+        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.MasterNbInExtViewVerifier(
+            ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+
+    result =
+        ClusterStateVerifier
+            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
+                CLUSTER_NAME));
+    Assert.assertTrue(result);
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    _manager.disconnect();
+    _controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      _participants[i].syncStop();
+    }
+  }
+
+  @Test
+  public void testUpdateQueueConfig() throws InterruptedException {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuild = TaskTestUtil.buildRecurrentJobQueue(queueName, 0, 600000);
+    // Create and Enqueue jobs
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i <= 1; i++) {
+      String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
+
+      JobConfig.Builder jobConfig =
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+              .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+              .setTargetPartitionStates(Sets.newHashSet(targetPartition));
+      String jobName = targetPartition.toLowerCase() + "Job" + i;
+      queueBuild.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuild.build());
+
+    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+
+    WorkflowConfig workflowConfig = TaskUtil.getWorkflowCfg(_manager, queueName);
+    WorkflowConfig.Builder configBuilder = new WorkflowConfig.Builder(workflowConfig);
+
+    Calendar startTime = Calendar.getInstance();
+    startTime.set(Calendar.SECOND, startTime.get(Calendar.SECOND) + 1);
+
+    ScheduleConfig scheduleConfig =
+        ScheduleConfig.recurringFromDate(startTime.getTime(), TimeUnit.MINUTES, 2);
+
+    configBuilder.setScheduleConfig(scheduleConfig);
+
+    // ensure current schedule is started
+    String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
+    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.IN_PROGRESS);
+
+    _driver.updateWorkflow(queueName, configBuilder.build());
+
+    // ensure current schedule is completed
+    TaskTestUtil.pollForWorkflowState(_manager, scheduledQueue, TaskState.COMPLETED);
+
+    Thread.sleep(1000);
+
+    wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
+    scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
+    WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_manager, scheduledQueue);
+
+    Calendar configStartTime = Calendar.getInstance();
+    configStartTime.setTime(wCfg.getStartTime());
+
+    Assert.assertTrue(
+        (startTime.get(Calendar.HOUR_OF_DAY) == configStartTime.get(Calendar.HOUR_OF_DAY) &&
+            startTime.get(Calendar.MINUTE) == configStartTime.get(Calendar.MINUTE) &&
+            startTime.get(Calendar.SECOND) == configStartTime.get(Calendar.SECOND)));
+  }
+}
+


[16/33] helix git commit: Add TaskTestBase and refactor 2 tests 1. Add TaskTestBase for common initialization. 2. Refactor 2 tests to test the test base.

Posted by lx...@apache.org.
Add TaskTestBase and refactor 2 tests
1. Add TaskTestBase for common initialization.
2. Refactor 2 tests to test the test base.


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

Branch: refs/heads/helix-0.6.x
Commit: ead83012ee07610859fabdc73712cf4532473e88
Parents: 1f683b8
Author: Junkai Xue <jx...@linkedin.com>
Authored: Fri Apr 8 13:40:04 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:58:44 2016 -0700

----------------------------------------------------------------------
 .../helix/integration/task/TaskTestBase.java    | 120 ++++++++++++++
 .../task/TestDisableJobExternalView.java        | 115 +-------------
 .../task/TestJobFailureDependence.java          | 158 ++++---------------
 3 files changed, 154 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ead83012/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
new file mode 100644
index 0000000..50baa14
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
@@ -0,0 +1,120 @@
+package org.apache.helix.integration.task;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.task.Task;
+import org.apache.helix.task.TaskCallbackContext;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+
+public class TaskTestBase extends ZkIntegrationTestBase {
+  protected int _numNodes = 5;
+  protected int _startPort = 12918;
+  protected int _numParitions = 20;
+  protected int _numReplicas = 3;
+  protected int _numDbs = 1;
+
+  protected ClusterControllerManager _controller;
+
+  protected HelixManager _manager;
+  protected TaskDriver _driver;
+  protected ClusterSetup _setupTool;
+
+  protected List<String> _testDbs = new ArrayList<String>();
+
+  protected final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
+  protected final MockParticipantManager[] _participants = new MockParticipantManager[_numNodes];
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+
+    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool.addCluster(CLUSTER_NAME, true);
+    for (int i = 0; i < _numNodes; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
+      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    }
+
+    // Set up target db
+    if (_numDbs > 1) {
+      for (int i = 0; i < _numDbs; i++) {
+        String db = WorkflowGenerator.DEFAULT_TGT_DB + i;
+        _setupTool
+            .addResourceToCluster(CLUSTER_NAME, db, _numParitions + 10 * i, MASTER_SLAVE_STATE_MODEL,
+                IdealState.RebalanceMode.FULL_AUTO.toString());
+        _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _numReplicas);
+        _testDbs.add(db);
+      }
+    } else {
+      _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions, MASTER_SLAVE_STATE_MODEL);
+      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numReplicas);
+    }
+
+    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
+    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
+      @Override public Task createNewTask(TaskCallbackContext context) {
+        return new MockTask(context);
+      }
+    });
+
+    // start dummy participants
+    for (int i = 0; i < _numNodes; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+
+      // Register a Task state model factory.
+      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
+      stateMachine.registerStateModelFactory("Task",
+          new TaskStateModelFactory(_participants[i], taskFactoryReg));
+      _participants[i].syncStart();
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // create cluster manager
+    _manager = HelixManagerFactory
+        .getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR, ZK_ADDR);
+    _manager.connect();
+    _driver = new TaskDriver(_manager);
+
+    boolean result = ClusterStateVerifier.verifyByZkCallback(
+        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    _manager.disconnect();
+
+    for (int i = 0; i < _numNodes; i++) {
+      _participants[i].syncStop();
+    }
+
+    _controller.syncStop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/ead83012/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
index f673f7b..4563e70 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
@@ -19,129 +19,26 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import com.google.common.collect.Sets;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import com.google.common.collect.Sets;
 
-public class TestDisableJobExternalView extends ZkIntegrationTestBase {
+public class TestDisableJobExternalView extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestDisableJobExternalView.class);
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
-  private static final String TGT_DB = "TestDB";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target db
-    setupTool.addResourceToCluster(CLUSTER_NAME, TGT_DB, NUM_PARTITIONS, MASTER_SLAVE_STATE_MODEL);
-    setupTool.rebalanceStorageCluster(CLUSTER_NAME, TGT_DB, NUM_REPLICAS);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override
-      public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.MasterNbInExtViewVerifier(
-            ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    for (int i = 0; i < n; i++) {
-      _participants[i].syncStop();
-    }
-    _controller.syncStop();
-  }
-
 
   @Test
   public void testJobsDisableExternalView() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/ead83012/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
index 9e2456c..d4f6dbb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
@@ -19,129 +19,27 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import com.google.common.collect.Sets;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.WorkflowConfig;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import com.google.common.collect.Sets;
 
-public class TestJobFailureDependence extends ZkIntegrationTestBase {
+public class TestJobFailureDependence extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestJobFailureDependence.class);
-  private static final int num_nodes = 5;
-  private static final int num_dbs = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[num_nodes];
-  private ClusterControllerManager _controller;
-  private ClusterSetup _setupTool;
-
-  private List<String> _test_dbs = new ArrayList<String>();
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
 
   @BeforeClass
   public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < num_nodes; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target dbs
-    for (int i = 0; i < num_dbs; i++) {
-      String db = "TestDB" + i;
-      _setupTool
-          .addResourceToCluster(CLUSTER_NAME, db, NUM_PARTITIONS + 10 * i, MASTER_SLAVE_STATE_MODEL,
-              IdealState.RebalanceMode.FULL_AUTO.toString());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, NUM_REPLICAS);
-      _test_dbs.add(db);
-    }
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < num_nodes; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task",
-          new TaskStateModelFactory(_participants[i], taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager = HelixManagerFactory
-        .getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR, ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < num_nodes; i++) {
-      _participants[i].syncStop();
-    }
+    _numDbs = 5;
+    super.beforeClass();
   }
 
   @Test
@@ -153,20 +51,20 @@ public class TestJobFailureDependence extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 100);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
 
     // all jobs after failed job should fail too.
-    for (int i = 2; i < num_dbs; i++) {
+    for (int i = 2; i < _numDbs; i++) {
       String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(i));
       TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.FAILED);
     }
@@ -181,17 +79,17 @@ public class TestJobFailureDependence extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(2));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
@@ -207,22 +105,22 @@ public class TestJobFailureDependence extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 100);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
     String namedSpaceJob2 = String.format("%s_%s", queueName, currentJobNames.get(2));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob2, TaskState.FAILED);
 
     // all jobs after failed job should complete.
-    for (int i = 3; i < num_dbs; i++) {
+    for (int i = 3; i < _numDbs; i++) {
       String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(i));
       TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.COMPLETED);
     }
@@ -237,17 +135,17 @@ public class TestJobFailureDependence extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 3);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(1));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
@@ -264,11 +162,11 @@ public class TestJobFailureDependence extends ZkIntegrationTestBase {
     _driver.updateWorkflow(queueName, configBuilder.build());
     _driver.stop(queueName);
 
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       _driver.enqueueJob(queueName, jobName, jobConfig);
     }


[13/33] helix git commit: Fix a task unit test.

Posted by lx...@apache.org.
Fix a task unit test.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/57bfc4a2
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/57bfc4a2
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/57bfc4a2

Branch: refs/heads/helix-0.6.x
Commit: 57bfc4a211b2f3454c8f00fa3f26262cd1d3922b
Parents: aeb6f3e
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Mar 22 09:08:09 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:55:55 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/integration/task/TestTaskRebalancer.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/57bfc4a2/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index 74b10fd..9df920b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -183,7 +183,7 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
 
     // Wait for job to finish and expire
     TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
-    Thread.sleep(expiry);
+    Thread.sleep(expiry + 100);
 
     // Ensure workflow config and context were cleaned up by now
     Assert.assertFalse(_manager.getHelixPropertyStore().exists(workflowPropStoreKey,


[26/33] helix git commit: Refactor redundant code TestTaskRebalancerRetryLimit

Posted by lx...@apache.org.
Refactor redundant code TestTaskRebalancerRetryLimit

RB=715304
G=nuage-reviewers
R=lxia,cji
A=lxia


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

Branch: refs/heads/helix-0.6.x
Commit: 2f62cd1fc60eab43a2250224576b28742f023a73
Parents: e8ecdf3
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Apr 28 15:24:03 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:18:03 2016 -0700

----------------------------------------------------------------------
 .../task/TestTaskRebalancerRetryLimit.java      | 88 ++------------------
 1 file changed, 5 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/2f62cd1f/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
index a277358..d677920 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
@@ -19,97 +19,33 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskPartitionState;
 import org.apache.helix.task.TaskResult;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
 import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.Workflow;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 /**
  * Test task will be retried up to MaxAttemptsPerTask {@see HELIX-562}
  */
 public class TestTaskRebalancerRetryLimit extends TaskTestBase {
 
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    ClusterSetup setup = new ClusterSetup(_gZkClient);
-    setup.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < _numNodes; i++) {
-      String instanceName = "localhost_" + (12918 + i);
-      setup.addInstanceToCluster(CLUSTER_NAME, instanceName);
-    }
-
-    // Set up target db
-    setup.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions, "MasterSlave");
-    setup.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numReplicas);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put("ErrorTask", new TaskFactory() {
-      @Override
-      public Task createNewTask(TaskCallbackContext context) {
-        return new ErrorTask();
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < _numNodes; i++) {
-      String instanceName = "localhost_" + (12918 + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = "controller";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-
   @Test public void test() throws Exception {
     String jobResource = TestHelper.getTestMethodName();
 
     JobConfig.Builder jobBuilder = JobConfig.Builder.fromMap(WorkflowGenerator.DEFAULT_JOB_CONFIG);
     jobBuilder.setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG)
-        .setMaxAttemptsPerTask(2).setCommand("ErrorTask").setFailureThreshold(Integer.MAX_VALUE);
+        .setMaxAttemptsPerTask(2).setCommand(MockTask.TASK_COMMAND)
+        .setFailureThreshold(Integer.MAX_VALUE)
+        .setJobCommandConfigMap(ImmutableMap.of(MockTask.THROW_EXCEPTION, "true"));
 
     Workflow flow =
         WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
@@ -128,18 +64,4 @@ public class TestTaskRebalancerRetryLimit extends TaskTestBase {
       }
     }
   }
-
-  private static class ErrorTask implements Task {
-    public ErrorTask() {
-    }
-
-    @Override
-    public TaskResult run() {
-      throw new RuntimeException("IGNORABLE exception: test throw exception from task");
-    }
-
-    @Override
-    public void cancel() {
-    }
-  }
 }


[18/33] helix git commit: Refactor tests with TaskTestBase and remove duplicated code.

Posted by lx...@apache.org.
Refactor tests with TaskTestBase and remove duplicated code.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/99a40083
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/99a40083
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/99a40083

Branch: refs/heads/helix-0.6.x
Commit: 99a40083ab7bc1e1480d66107fff83f0479fa068
Parents: 9f80206
Author: Junkai Xue <jx...@linkedin.com>
Authored: Tue Apr 12 11:48:54 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 15:02:05 2016 -0700

----------------------------------------------------------------------
 .../helix/integration/task/TestGenericJobs.java |  99 +------------
 .../task/TestIndependentTaskRebalancer.java     |  19 +--
 .../integration/task/TestRecurringJobQueue.java | 113 +--------------
 .../task/TestRunJobsWithMissingTarget.java      | 138 +++----------------
 .../integration/task/TestTaskRebalancer.java    | 102 +-------------
 .../task/TestTaskRebalancerFailover.java        |  97 +------------
 .../task/TestTaskRebalancerParallel.java        | 106 +-------------
 .../task/TestTaskRebalancerRetryLimit.java      |  45 ++----
 .../task/TestTaskRebalancerStopResume.java      | 106 +-------------
 .../integration/task/TestUpdateWorkflow.java    | 120 +---------------
 10 files changed, 58 insertions(+), 887 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
index d96acd9..426bade 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
@@ -19,110 +19,19 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskConfig;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class TestGenericJobs extends ZkIntegrationTestBase {
+public class TestGenericJobs extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestGenericJobs.class);
-  private static final int num_nodes = 5;
-  private static final int START_PORT = 12918;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[num_nodes];
-  private ClusterControllerManager _controller;
-  private ClusterSetup _setupTool;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
-
-  @BeforeClass public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < num_nodes; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < num_nodes; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task",
-          new TaskStateModelFactory(_participants[i], taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager = HelixManagerFactory
-        .getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR, ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < num_nodes; i++) {
-      _participants[i].syncStop();
-    }
-    _setupTool.deleteCluster(CLUSTER_NAME);
-  }
 
   @Test public void testGenericJobs() throws Exception {
     String queueName = TestHelper.getTestMethodName();

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
index 046281e..0e598c1 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
@@ -58,17 +58,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
+public class TestIndependentTaskRebalancer extends TaskTestBase {
   private Set<String> _invokedClasses = Sets.newHashSet();
   private Map<String, Integer> _runCounts = Maps.newHashMap();
 
-  private HelixManager _manager;
-  private TaskDriver _driver;
 
   @BeforeClass
   public void beforeClass() throws Exception {
@@ -80,14 +73,14 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     // Setup cluster and instances
     ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
     setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+    for (int i = 0; i < _numNodes; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
       setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     // start dummy participants
-    for (int i = 0; i < n; i++) {
-      final String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+    for (int i = 0; i < _numNodes; i++) {
+      final String instanceName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
 
       // Set task callbacks
       Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
@@ -226,7 +219,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(2);
     Map<String, String> taskConfigMap = Maps.newHashMap(
-        ImmutableMap.of("fail", "" + true, "failInstance", PARTICIPANT_PREFIX + '_' + START_PORT));
+        ImmutableMap.of("fail", "" + true, "failInstance", PARTICIPANT_PREFIX + '_' + _startPort));
     TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap, false);
     taskConfigs.add(taskConfig1);
     Map<String, String> jobCommandMap = Maps.newHashMap();

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index b2e61ca..8262b9b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -20,140 +20,29 @@ package org.apache.helix.integration.task;
  */
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.TargetState;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
-public class TestRecurringJobQueue extends ZkIntegrationTestBase {
+public class TestRecurringJobQueue extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestRecurringJobQueue.class);
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
   private static final String TIMEOUT_CONFIG = "Timeout";
-  private static final String TGT_DB = "TestDB";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
-  private ZKHelixDataAccessor _accessor;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    _accessor =
-        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target db
-    setupTool.addResourceToCluster(CLUSTER_NAME, TGT_DB, NUM_PARTITIONS, MASTER_SLAVE_STATE_MODEL);
-    setupTool.rebalanceStorageCluster(CLUSTER_NAME, TGT_DB, NUM_REPLICAS);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override
-      public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.MasterNbInExtViewVerifier(
-            ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < n; i++) {
-      _participants[i].syncStop();
-    }
-  }
-
-
 
   @Test
   public void deleteRecreateRecurrentQueue() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index bd05f81..5a07942 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -19,128 +19,26 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import com.google.common.collect.Sets;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import com.google.common.collect.Sets;
 
-public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
+public class TestRunJobsWithMissingTarget extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestRunJobsWithMissingTarget.class);
-  private static final int num_nodes = 5;
-  private static final int num_dbs = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[num_nodes];
-  private ClusterControllerManager _controller;
-  private ClusterSetup _setupTool;
-
-  private List<String> _test_dbs = new ArrayList<String>();
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
 
   @BeforeClass
   public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < num_nodes; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target dbs
-    for (int i = 0; i < num_dbs; i++) {
-      String db = "TestDB" + i;
-      _setupTool
-          .addResourceToCluster(CLUSTER_NAME, db, NUM_PARTITIONS + 10 * i, MASTER_SLAVE_STATE_MODEL,
-              IdealState.RebalanceMode.FULL_AUTO.toString());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, NUM_REPLICAS);
-      _test_dbs.add(db);
-    }
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < num_nodes; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task",
-          new TaskStateModelFactory(_participants[i], taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager = HelixManagerFactory
-        .getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR, ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < num_nodes; i++) {
-      _participants[i].syncStop();
-    }
-    _setupTool.deleteCluster(CLUSTER_NAME);
+    _numDbs = 5;
+    super.beforeClass();
   }
 
   @Test
@@ -152,17 +50,17 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
           new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(
-              _test_dbs.get(i))
+              _testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
 
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(1));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
     _driver.start(queueBuilder.build());
 
     String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(1));
@@ -181,11 +79,11 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName, 0, 3);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setIgnoreDependentJobFailure(true);
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
@@ -210,17 +108,17 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    for (int i = 0; i < num_dbs; i++) {
+    for (int i = 0; i < _numDbs; i++) {
       JobConfig.Builder jobConfig =
-          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_test_dbs.get(i))
+          new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(_testDbs.get(i))
               .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
-      String jobName = "job" + _test_dbs.get(i);
+      String jobName = "job" + _testDbs.get(i);
       queueBuilder.enqueueJob(jobName, jobConfig);
       currentJobNames.add(jobName);
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(0));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(0));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(0));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index 9df920b..f5a3441 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -19,42 +19,26 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.AccessOption;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobDag;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskConstants;
 import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskPartitionState;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
 import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.Workflow;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import com.google.common.base.Joiner;
@@ -62,88 +46,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
-public class TestTaskRebalancer extends ZkIntegrationTestBase {
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
+public class TestTaskRebalancer extends TaskTestBase {
   private static final String TIMEOUT_CONFIG = "Timeout";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target db
-    setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_PARTITIONS,
-        MASTER_SLAVE_STATE_MODEL);
-    setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_REPLICAS);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override
-      public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    // _controller = null;
-    for (int i = 0; i < n; i++) {
-      _participants[i].syncStop();
-      // _participants[i] = null;
-    }
-  }
 
   @Test
   public void basic() throws Exception {
@@ -214,7 +118,7 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
 
     // Ensure all partitions are completed individually
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
-    for (int i = 0; i < NUM_PARTITIONS; i++) {
+    for (int i = 0; i < _numParitions; i++) {
       Assert.assertEquals(ctx.getPartitionState(i), TaskPartitionState.COMPLETED);
       Assert.assertEquals(ctx.getPartitionNumAttempts(i), 1);
     }
@@ -287,7 +191,7 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     // Check that all partitions timed out up to maxAttempts
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     int maxAttempts = 0;
-    for (int i = 0; i < NUM_PARTITIONS; i++) {
+    for (int i = 0; i < _numParitions; i++) {
       TaskPartitionState state = ctx.getPartitionState(i);
       if (state != null) {
         Assert.assertEquals(state, TaskPartitionState.TIMED_OUT);

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
index 8051b2f..9d98ba9 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
@@ -19,118 +19,27 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobDag;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.WorkflowConfig;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.Sets;
 
-public class TestTaskRebalancerFailover extends ZkUnitTestBase {
+public class TestTaskRebalancerFailover extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestTaskRebalancerFailover.class);
 
-  private final String _clusterName = TestHelper.getTestClassName();
-  private static final int _n = 5;
-  private static final int _p = 20;
-  private static final int _r = 3;
-  private final MockParticipantManager[] _participants = new MockParticipantManager[_n];
-  private ClusterControllerManager _controller;
-  private HelixManager _manager;
-  private TaskDriver _driver;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    ClusterSetup setup = new ClusterSetup(_gZkClient);
-    setup.addCluster(_clusterName, true);
-    for (int i = 0; i < _n; i++) {
-      String instanceName = "localhost_" + (12918 + i);
-      setup.addInstanceToCluster(_clusterName, instanceName);
-    }
-
-    // Set up target db
-    setup.addResourceToCluster(_clusterName, WorkflowGenerator.DEFAULT_TGT_DB, _p, "MasterSlave");
-    setup.rebalanceStorageCluster(_clusterName, WorkflowGenerator.DEFAULT_TGT_DB, _r);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override
-      public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < _n; i++) {
-      String instanceName = "localhost_" + (12918 + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, _clusterName, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = "controller";
-    _controller = new ClusterControllerManager(ZK_ADDR, _clusterName, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(_clusterName, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                _clusterName));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < _n; i++) {
-      if (_participants[i] != null && _participants[i].isConnected()) {
-        _participants[i].syncStop();
-      }
-    }
-  }
-
   @Test
   public void test() throws Exception {
     String queueName = TestHelper.getTestMethodName();
@@ -159,7 +68,7 @@ public class TestTaskRebalancerFailover extends ZkUnitTestBase {
         accessor.getProperty(keyBuilder.externalView(WorkflowGenerator.DEFAULT_TGT_DB));
     JobContext ctx = _driver.getJobContext(namespacedJob1);
     Set<String> failOverPartitions = Sets.newHashSet();
-    for (int p = 0; p < _p; p++) {
+    for (int p = 0; p < _numParitions; p++) {
       String instanceName = ctx.getAssignedParticipant(p);
       Assert.assertNotNull(instanceName);
       String partitionName = ctx.getTargetForPartition(p);
@@ -185,7 +94,7 @@ public class TestTaskRebalancerFailover extends ZkUnitTestBase {
     // tasks previously assigned to localhost_12918 should be re-scheduled on new master
     ctx = _driver.getJobContext(namespacedJob2);
     ev = accessor.getProperty(keyBuilder.externalView(WorkflowGenerator.DEFAULT_TGT_DB));
-    for (int p = 0; p < _p; p++) {
+    for (int p = 0; p < _numParitions; p++) {
       String partitionName = ctx.getTargetForPartition(p);
       Assert.assertNotNull(partitionName);
       if (failOverPartitions.contains(partitionName)) {

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
index b091748..dbc4154 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
@@ -20,121 +20,23 @@ package org.apache.helix.integration.task;
  */
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
-import org.apache.helix.task.TaskStateModelFactory;
 import org.apache.helix.task.WorkflowConfig;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestTaskRebalancerParallel extends ZkIntegrationTestBase {
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final List<String> testDbNames =
-      Arrays.asList("TestDB_1", "TestDB_2", "TestDB_3", "TestDB_4");
-
-
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
+public class TestTaskRebalancerParallel extends TaskTestBase {
 
   @BeforeClass
   public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    for (String testDbName : testDbNames) {
-      setupTool.addResourceToCluster(CLUSTER_NAME, testDbName, NUM_PARTITIONS,
-          MASTER_SLAVE_STATE_MODEL);
-      setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDbName, NUM_REPLICAS);
-    }
-
-    // start dummy participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      final long delay = (i + 1) * 1000L;
-      Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-      taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-        @Override
-        public Task createNewTask(TaskCallbackContext context) {
-          return new MockTask(context);
-        }
-      });
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task",
-          new TaskStateModelFactory(_participants[i], taskFactoryReg));
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    // _controller = null;
-    for (int i = 0; i < n; i++) {
-      _participants[i].syncStop();
-      // _participants[i] = null;
-    }
+   _numDbs = 4;
+    super.beforeClass();
   }
 
   @Test public void test() throws Exception {
@@ -151,7 +53,7 @@ public class TestTaskRebalancerParallel extends ZkIntegrationTestBase {
     _driver.createQueue(queue);
 
     List<JobConfig.Builder> jobConfigBuilders = new ArrayList<JobConfig.Builder>();
-    for (String testDbName : testDbNames) {
+    for (String testDbName : _testDbs) {
       jobConfigBuilders.add(
           new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(testDbName)
               .setTargetPartitionStates(Collections.singleton("SLAVE")));

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
index e576304..a277358 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
@@ -22,11 +22,9 @@ package org.apache.helix.integration.task;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.participant.StateMachineEngine;
@@ -45,35 +43,26 @@ import org.apache.helix.task.Workflow;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 /**
  * Test task will be retried up to MaxAttemptsPerTask {@see HELIX-562}
  */
-public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
-  private final String _clusterName = TestHelper.getTestClassName();
-  private static final int _n = 5;
-  private static final int _p = 20;
-  private static final int _r = 3;
-  private final MockParticipantManager[] _participants = new MockParticipantManager[_n];
-  private ClusterControllerManager _controller;
-  private HelixManager _manager;
-  private TaskDriver _driver;
+public class TestTaskRebalancerRetryLimit extends TaskTestBase {
 
   @BeforeClass
   public void beforeClass() throws Exception {
     ClusterSetup setup = new ClusterSetup(_gZkClient);
-    setup.addCluster(_clusterName, true);
-    for (int i = 0; i < _n; i++) {
+    setup.addCluster(CLUSTER_NAME, true);
+    for (int i = 0; i < _numNodes; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      setup.addInstanceToCluster(_clusterName, instanceName);
+      setup.addInstanceToCluster(CLUSTER_NAME, instanceName);
     }
 
     // Set up target db
-    setup.addResourceToCluster(_clusterName, WorkflowGenerator.DEFAULT_TGT_DB, _p, "MasterSlave");
-    setup.rebalanceStorageCluster(_clusterName, WorkflowGenerator.DEFAULT_TGT_DB, _r);
+    setup.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions, "MasterSlave");
+    setup.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numReplicas);
 
     Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
     taskFactoryReg.put("ErrorTask", new TaskFactory() {
@@ -84,9 +73,9 @@ public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
     });
 
     // start dummy participants
-    for (int i = 0; i < _n; i++) {
+    for (int i = 0; i < _numNodes; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, _clusterName, instanceName);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
 
       // Register a Task state model factory.
       StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
@@ -97,12 +86,12 @@ public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
 
     // start controller
     String controllerName = "controller";
-    _controller = new ClusterControllerManager(ZK_ADDR, _clusterName, controllerName);
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     // create cluster manager
     _manager =
-        HelixManagerFactory.getZKHelixManager(_clusterName, "Admin", InstanceType.ADMINISTRATOR,
+        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
             ZK_ADDR);
     _manager.connect();
     _driver = new TaskDriver(_manager);
@@ -110,20 +99,10 @@ public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
     boolean result =
         ClusterStateVerifier
             .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                _clusterName));
+                CLUSTER_NAME));
     Assert.assertTrue(result);
   }
 
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < _n; i++) {
-      if (_participants[i] != null && _participants[i].isConnected()) {
-        _participants[i].syncStop();
-      }
-    }
-  }
 
   @Test public void test() throws Exception {
     String jobResource = TestHelper.getTestMethodName();
@@ -141,7 +120,7 @@ public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
     TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
 
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
-    for (int i = 0; i < _p; i++) {
+    for (int i = 0; i < _numParitions; i++) {
       TaskPartitionState state = ctx.getPartitionState(i);
       if (state != null) {
         Assert.assertEquals(state, TaskPartitionState.TASK_ERROR);

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
index 30cb460..e92a129 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
@@ -21,141 +21,39 @@ package org.apache.helix.integration.task;
 
 import java.util.ArrayList;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.Lists;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.JobDag;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.ScheduleConfig;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskConstants;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.Workflow;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
-public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
+public class TestTaskRebalancerStopResume extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestTaskRebalancerStopResume.class);
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
   private static final String TIMEOUT_CONFIG = "Timeout";
-  private static final String TGT_DB = "TestDB";
   private static final String JOB_RESOURCE = "SomeJob";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target db
-    setupTool.addResourceToCluster(CLUSTER_NAME, TGT_DB, NUM_PARTITIONS, MASTER_SLAVE_STATE_MODEL);
-    setupTool.rebalanceStorageCluster(CLUSTER_NAME, TGT_DB, NUM_REPLICAS);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg
-        .put(MockTask.TASK_COMMAND, new TaskFactory() {
-          @Override public Task createNewTask(TaskCallbackContext context) {
-            return new MockTask(context);
-          }
-        });
-
-    // start dummy participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.MasterNbInExtViewVerifier(
-            ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < n; i++) {
-      _participants[i].syncStop();
-    }
-  }
 
   @Test public void stopAndResume() throws Exception {
     Map<String, String> commandConfig = ImmutableMap.of(TIMEOUT_CONFIG, String.valueOf(100));

http://git-wip-us.apache.org/repos/asf/helix/blob/99a40083/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
index 2e53b36..b43c49e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
@@ -19,135 +19,25 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
-import com.google.common.collect.Sets;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
+import java.util.Calendar;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.ScheduleConfig;
 import org.apache.helix.task.TargetState;
-import org.apache.helix.task.Task;
-import org.apache.helix.task.TaskCallbackContext;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskStateModelFactory;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
+import com.google.common.collect.Sets;
 
-public class TestUpdateWorkflow extends ZkIntegrationTestBase {
+public class TestUpdateWorkflow extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestUpdateWorkflow.class);
-  private static final int n = 5;
-  private static final int START_PORT = 12918;
-  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
-  private static final String TIMEOUT_CONFIG = "Timeout";
-  private static final int NUM_PARTITIONS = 20;
-  private static final int NUM_REPLICAS = 3;
-  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
-
-  private HelixManager _manager;
-  private TaskDriver _driver;
-  private ZKHelixDataAccessor _accessor;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-
-    _accessor =
-        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
-    for (int i = 0; i < n; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-    }
-
-    // Set up target db
-    setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_PARTITIONS,
-        MASTER_SLAVE_STATE_MODEL);
-    setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_REPLICAS);
-
-    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
-    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
-      @Override
-      public Task createNewTask(TaskCallbackContext context) {
-        return new MockTask(context);
-      }
-    });
-
-    // start dummy participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-
-      // Register a Task state model factory.
-      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
-      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
-          taskFactoryReg));
-
-      _participants[i].syncStart();
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    // create cluster manager
-    _manager =
-        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
-            ZK_ADDR);
-    _manager.connect();
-
-    _driver = new TaskDriver(_manager);
-
-    boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.MasterNbInExtViewVerifier(
-            ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-
-    result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    _manager.disconnect();
-    _controller.syncStop();
-    for (int i = 0; i < n; i++) {
-      _participants[i].syncStop();
-    }
-  }
 
   @Test
   public void testUpdateRunningQueue() throws InterruptedException {


[09/33] helix git commit: Fix task framework unit test failure.

Posted by lx...@apache.org.
Fix task framework unit test failure.


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

Branch: refs/heads/helix-0.6.x
Commit: 1dad0b805338a8dae72e0661ac1a803b30808623
Parents: be66024
Author: Lei Xia <lx...@linkedin.com>
Authored: Mon Mar 7 15:15:10 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:45:18 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/WorkflowContext.java    | 13 +++++++++++++
 .../apache/helix/integration/task/TaskTestUtil.java    |  3 ++-
 .../integration/task/TestRunJobsWithMissingTarget.java |  9 ++++-----
 3 files changed, 19 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/1dad0b80/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
index 5fc7e66..0e0a283 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
@@ -19,6 +19,7 @@ package org.apache.helix.task;
  * under the License.
  */
 
+import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -82,6 +83,18 @@ public class WorkflowContext extends HelixProperty {
     return TaskState.valueOf(s);
   }
 
+  public Map<String, TaskState> getJobStates() {
+    Map<String, TaskState> jobStates = new HashMap<String, TaskState>();
+    Map<String, String> stateFieldMap = _record.getMapField(JOB_STATES);
+    if (stateFieldMap != null) {
+      for (Map.Entry<String, String> state : stateFieldMap.entrySet()) {
+        jobStates.put(state.getKey(), TaskState.valueOf(state.getValue()));
+      }
+    }
+
+    return jobStates;
+  }
+
   public void setStartTime(long t) {
     _record.setSimpleField(START_TIME, String.valueOf(t));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/1dad0b80/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index 011f532..422ec88 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -112,7 +112,8 @@ public class TaskTestUtil {
     Assert.assertNotNull(ctx, "Empty job context");
     TaskState jobState = ctx.getJobState(jobName);
     Assert.assertTrue(allowedStates.contains(jobState),
-        "expect job states: " + allowedStates + " actual job state: " + jobState);
+        "expect job " + jobName + " is in states: " + allowedStates + " actual job state: "
+            + jobState + " all other job states in the workflow: " + ctx.getJobStates().entrySet());
   }
 
   public static void pollForEmptyJobState(final TaskDriver driver, final String workflowName,

http://git-wip-us.apache.org/repos/asf/helix/blob/1dad0b80/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index 7eeb3f4..bd05f81 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -162,17 +162,17 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
       currentJobNames.add(jobName);
     }
 
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(2));
+    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(1));
     _driver.start(queueBuilder.build());
 
-    String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(2));
+    String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(1));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.FAILED);
     TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
 
     _driver.delete(queueName);
   }
 
-  @Test
+  @Test(dependsOnMethods = "testJobFailsWithMissingTarget")
   public void testJobContinueUponParentJobFailure() throws Exception {
     String queueName = TestHelper.getTestMethodName();
 
@@ -191,7 +191,6 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _test_dbs.get(1));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
     TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
@@ -202,7 +201,7 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
     _driver.delete(queueName);
   }
 
-  @Test
+  @Test(dependsOnMethods = "testJobContinueUponParentJobFailure")
   public void testJobFailsWithMissingTargetInRunning() throws Exception {
     String queueName = TestHelper.getTestMethodName();
 


[02/33] helix git commit: More fixes and cleanup on task unit tests.

Posted by lx...@apache.org.
More fixes and cleanup on task unit tests.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/03b90012
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/03b90012
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/03b90012

Branch: refs/heads/helix-0.6.x
Commit: 03b90012939e99eb3556c715ff8b8eaab710bb79
Parents: 760f8e3
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Jan 20 15:33:20 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:34:12 2016 -0700

----------------------------------------------------------------------
 .../task/TestDisableJobExternalView.java        | 210 +++++++++++++++++++
 .../integration/task/TestRecurringJobQueue.java |  74 +------
 .../task/TestRunJobsWithMissingTarget.java      |   7 +-
 .../integration/task/TestTaskRebalancer.java    |   3 +-
 .../task/TestTaskRebalancerFailover.java        |   2 +-
 .../task/TestTaskRebalancerParallel.java        |   3 +-
 .../task/TestTaskRebalancerRetryLimit.java      |   2 +-
 .../task/TestTaskRebalancerStopResume.java      |  13 +-
 8 files changed, 222 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
new file mode 100644
index 0000000..b23e268
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
@@ -0,0 +1,210 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import com.google.common.collect.Sets;
+import org.apache.helix.ExternalViewChangeListener;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.TestHelper;
+import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.Task;
+import org.apache.helix.task.TaskCallbackContext;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class TestDisableJobExternalView extends ZkIntegrationTestBase {
+  private static final Logger LOG = Logger.getLogger(TestDisableJobExternalView.class);
+  private static final int n = 5;
+  private static final int START_PORT = 12918;
+  private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
+  private static final String TGT_DB = "TestDB";
+  private static final int NUM_PARTITIONS = 20;
+  private static final int NUM_REPLICAS = 3;
+  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
+  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
+  private ClusterControllerManager _controller;
+
+  private HelixManager _manager;
+  private TaskDriver _driver;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+
+    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
+    setupTool.addCluster(CLUSTER_NAME, true);
+    for (int i = 0; i < n; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    }
+
+    // Set up target db
+    setupTool.addResourceToCluster(CLUSTER_NAME, TGT_DB, NUM_PARTITIONS, MASTER_SLAVE_STATE_MODEL);
+    setupTool.rebalanceStorageCluster(CLUSTER_NAME, TGT_DB, NUM_REPLICAS);
+
+    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
+    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
+      @Override
+      public Task createNewTask(TaskCallbackContext context) {
+        return new MockTask(context);
+      }
+    });
+
+    // start dummy participants
+    for (int i = 0; i < n; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+
+      // Register a Task state model factory.
+      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
+      stateMachine.registerStateModelFactory("Task", new TaskStateModelFactory(_participants[i],
+          taskFactoryReg));
+
+      _participants[i].syncStart();
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // create cluster manager
+    _manager =
+        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR,
+            ZK_ADDR);
+    _manager.connect();
+
+    _driver = new TaskDriver(_manager);
+
+    boolean result =
+        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.MasterNbInExtViewVerifier(
+            ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+
+    result =
+        ClusterStateVerifier
+            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
+                CLUSTER_NAME));
+    Assert.assertTrue(result);
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    _manager.disconnect();
+    for (int i = 0; i < n; i++) {
+      _participants[i].syncStop();
+    }
+    _controller.syncStop();
+  }
+
+
+  @Test
+  public void testJobsDisableExternalView() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    ExternviewChecker externviewChecker = new ExternviewChecker();
+    _manager.addExternalViewChangeListener(externviewChecker);
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
+
+    JobConfig.Builder job1 = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+        .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
+
+    JobConfig.Builder job2 = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+        .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setDisableExternalView(true);
+
+    JobConfig.Builder job3 = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+        .setTargetPartitionStates(Sets.newHashSet("MASTER")).setDisableExternalView(false);
+
+    // enqueue jobs
+    queueBuilder.enqueueJob("job1", job1);
+    queueBuilder.enqueueJob("job2", job2);
+    queueBuilder.enqueueJob("job3", job3);
+
+    _driver.createQueue(queueBuilder.build());
+
+    // ensure all jobs are completed
+    String namedSpaceJob3 = String.format("%s_%s", queueName, "job3");
+    TaskTestUtil.pollForJobState(_manager, queueName, namedSpaceJob3, TaskState.COMPLETED);
+
+    Set<String> seenExternalViews = externviewChecker.getSeenExternalViews();
+    String namedSpaceJob1 = String.format("%s_%s", queueName, "job1");
+    String namedSpaceJob2 = String.format("%s_%s", queueName, "job2");
+
+    Assert.assertTrue(seenExternalViews.contains(namedSpaceJob1),
+        "Can not find external View for " + namedSpaceJob1 + "!");
+    Assert.assertTrue(!seenExternalViews.contains(namedSpaceJob2),
+        "External View for " + namedSpaceJob2 + " shoudld not exist!");
+    Assert.assertTrue(seenExternalViews.contains(namedSpaceJob3),
+        "Can not find external View for " + namedSpaceJob3 + "!");
+
+    _manager
+        .removeListener(new PropertyKey.Builder(CLUSTER_NAME).externalViews(), externviewChecker);
+  }
+
+  private static class ExternviewChecker implements ExternalViewChangeListener {
+    private Set<String> _seenExternalViews = new HashSet<String>();
+
+    @Override public void onExternalViewChange(List<ExternalView> externalViewList,
+        NotificationContext changeContext) {
+      for (ExternalView view : externalViewList) {
+        _seenExternalViews.add(view.getResourceName());
+      }
+    }
+
+    public Set<String> getSeenExternalViews() {
+      return _seenExternalViews;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index 4e21ef7..cb44f0e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -21,17 +21,13 @@ package org.apache.helix.integration.task;
 
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
-import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
-import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
@@ -40,7 +36,6 @@ import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.model.ExternalView;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
@@ -150,11 +145,11 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     for (int i = 0; i < n; i++) {
       _participants[i].syncStop();
     }
-    _manager.disconnect();
   }
 
 
@@ -372,73 +367,6 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
         String.format("%s_%s", scheduledQueue, jobNames.get(JOB_COUNTS - 1)));
   }
 
-  @Test
-  public void testJobsDisableExternalView() throws Exception {
-    String queueName = TestHelper.getTestMethodName();
-
-    ExternviewChecker externviewChecker = new ExternviewChecker();
-    _manager.addExternalViewChangeListener(externviewChecker);
-
-    // Create a queue
-    LOG.info("Starting job-queue: " + queueName);
-    JobQueue.Builder queueBuilder = TaskTestUtil.buildRecurrentJobQueue(queueName);
-
-    JobConfig.Builder job1 = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
-        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-        .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
-
-    JobConfig.Builder job2 = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
-        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-        .setTargetPartitionStates(Sets.newHashSet("SLAVE")).setDisableExternalView(true);
-
-    JobConfig.Builder job3 = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
-        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-        .setTargetPartitionStates(Sets.newHashSet("MASTER")).setDisableExternalView(false);
-
-    // enqueue both jobs
-    queueBuilder.enqueueJob("job1", job1);
-    queueBuilder.enqueueJob("job2", job2);
-    queueBuilder.enqueueJob("job3", job3);
-
-    _driver.createQueue(queueBuilder.build());
-
-    WorkflowContext wCtx = TaskTestUtil.pollForWorkflowContext(_manager, queueName);
-    String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-
-    // ensure all jobs are completed
-    String namedSpaceJob3 = String.format("%s_%s", scheduledQueue, "job3");
-    TaskTestUtil.pollForJobState(_manager, scheduledQueue, namedSpaceJob3, TaskState.COMPLETED);
-
-    Set<String> seenExternalViews = externviewChecker.getSeenExternalViews();
-    String namedSpaceJob1 = String.format("%s_%s", scheduledQueue, "job1");
-    String namedSpaceJob2 = String.format("%s_%s", scheduledQueue, "job2");
-
-    Assert.assertTrue(seenExternalViews.contains(namedSpaceJob1),
-        "Can not find external View for " + namedSpaceJob1 + "!");
-    Assert.assertTrue(!seenExternalViews.contains(namedSpaceJob2),
-        "External View for " + namedSpaceJob2 + " shoudld not exist!");
-    Assert.assertTrue(seenExternalViews.contains(namedSpaceJob3),
-        "Can not find external View for " + namedSpaceJob3 + "!");
-
-    _manager
-        .removeListener(new PropertyKey.Builder(CLUSTER_NAME).externalViews(), externviewChecker);
-  }
-
-  private static class ExternviewChecker implements ExternalViewChangeListener {
-    private Set<String> _seenExternalViews = new HashSet<String>();
-
-    @Override public void onExternalViewChange(List<ExternalView> externalViewList,
-        NotificationContext changeContext) {
-      for (ExternalView view : externalViewList) {
-        _seenExternalViews.add(view.getResourceName());
-      }
-    }
-
-    public Set<String> getSeenExternalViews() {
-      return _seenExternalViews;
-    }
-  }
-
   private void verifyJobDeleted(String queueName, String jobName) throws Exception {
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index d8d2b60..31e4325 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -35,10 +35,8 @@ import org.apache.helix.task.Task;
 import org.apache.helix.task.TaskCallbackContext;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskFactory;
-import org.apache.helix.task.TaskResult;
 import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskStateModelFactory;
-import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
@@ -48,8 +46,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -60,7 +56,6 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
   private static final int num_dbs = 5;
   private static final int START_PORT = 12918;
   private static final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
-  private static final String TIMEOUT_CONFIG = "Timeout";
   private static final int NUM_PARTITIONS = 20;
   private static final int NUM_REPLICAS = 3;
   private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
@@ -139,11 +134,11 @@ public class TestRunJobsWithMissingTarget extends ZkIntegrationTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     for (int i = 0; i < num_nodes; i++) {
       _participants[i].syncStop();
     }
-    _manager.disconnect();
   }
 
   @Test public void testJobFailsWithMissingTarget() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index 787ebcc..2d11f85 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -136,14 +136,13 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     // _controller = null;
     for (int i = 0; i < n; i++) {
       _participants[i].syncStop();
       // _participants[i] = null;
     }
-
-    _manager.disconnect();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
index 6f1c48e..a778dcd 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
@@ -122,13 +122,13 @@ public class TestTaskRebalancerFailover extends ZkUnitTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     for (int i = 0; i < _n; i++) {
       if (_participants[i] != null && _participants[i].isConnected()) {
         _participants[i].syncStop();
       }
     }
-    _manager.disconnect();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
index 5180a04..c9a0445 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
@@ -127,14 +127,13 @@ public class TestTaskRebalancerParallel extends ZkIntegrationTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     // _controller = null;
     for (int i = 0; i < n; i++) {
       _participants[i].syncStop();
       // _participants[i] = null;
     }
-
-    _manager.disconnect();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
index d25ffc5..8fec899 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
@@ -116,13 +116,13 @@ public class TestTaskRebalancerRetryLimit extends ZkIntegrationTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     for (int i = 0; i < _n; i++) {
       if (_participants[i] != null && _participants[i].isConnected()) {
         _participants[i].syncStop();
       }
     }
-    _manager.disconnect();
   }
 
   @Test public void test() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/03b90012/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
index 471d130..7a8d305 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
@@ -150,11 +150,11 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
 
   @AfterClass
   public void afterClass() throws Exception {
+    _manager.disconnect();
     _controller.syncStop();
     for (int i = 0; i < n; i++) {
       _participants[i].syncStop();
     }
-    _manager.disconnect();
   }
 
   @Test public void stopAndResume() throws Exception {
@@ -306,14 +306,14 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     _driver.resume(queueName);
 
     // ensure job 2 is started
-    TaskTestUtil.pollForJobState(_manager, queueName,
-        String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.IN_PROGRESS);
+    TaskTestUtil
+        .pollForJobState(_manager, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_manager, queueName,
-        String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.STOPPED);
+    TaskTestUtil
+        .pollForJobState(_manager, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.STOPPED);
     TaskTestUtil.pollForWorkflowState(_manager, queueName, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
@@ -334,8 +334,7 @@ public class TestTaskRebalancerStopResume extends ZkIntegrationTestBase {
     // add job 3 back
     JobConfig.Builder job =
         new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
-            .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
-            .setTargetPartitionStates(Sets.newHashSet("SLAVE"));
+            .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB).setTargetPartitionStates(Sets.newHashSet("SLAVE"));
     LOG.info("Enqueuing job: " + deletedJob2);
     _driver.enqueueJob(queueName, deletedJob2, job);
     currentJobNames.add(deletedJob2);


[23/33] helix git commit: Add unit tests to retrieve all workflows and job info from a cluster.

Posted by lx...@apache.org.
Add unit tests to retrieve all workflows and job info from a cluster.


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

Branch: refs/heads/helix-0.6.x
Commit: 2efa4485145ddfc8533c66c3e67077afe3a1ae9c
Parents: 4f7fe13
Author: Lei Xia <lx...@linkedin.com>
Authored: Thu Apr 21 10:58:49 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:17:13 2016 -0700

----------------------------------------------------------------------
 .../integration/task/TestRetrieveWorkflows.java | 26 +++++++-------------
 1 file changed, 9 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/2efa4485/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
index 786be7c..ddd6c85 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
@@ -18,13 +18,10 @@ package org.apache.helix.integration.task;
  * specific language governing permissions and limitations
  * under the License.
  */
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
-import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskState;
 import org.apache.helix.task.Workflow;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
@@ -36,33 +33,28 @@ import java.util.List;
 import java.util.Map;
 
 public class TestRetrieveWorkflows extends TaskTestBase {
-  @Test
-  public void testGetAllWorkflows() throws Exception {
+  @Test public void testGetAllWorkflows() throws Exception {
     List<Workflow> workflowList = new ArrayList<Workflow>();
-    for (int i = 0; i < 4; i++) {
-      Workflow workflow = WorkflowGenerator.generateDefaultRepeatedJobWorkflowBuilder(TestHelper.getTestMethodName() + i).build();
+    for (int i = 0; i < 2; i++) {
+      Workflow workflow = WorkflowGenerator
+          .generateDefaultRepeatedJobWorkflowBuilder(TestHelper.getTestMethodName() + i).build();
       _driver.start(workflow);
       workflowList.add(workflow);
     }
 
     for (Workflow workflow : workflowList) {
-      //TaskTestUtil.pollForWorkflowState(_driver, workflow.getName(), TaskState.COMPLETED);
+      TaskTestUtil.pollForWorkflowState(_driver, workflow.getName(), TaskState.COMPLETED);
     }
 
-    _manager = HelixManagerFactory
-        .getZKHelixManager("ESPRESSO_TEST_NUAGE", "Admin", InstanceType.ADMINISTRATOR, "zk-ei1-espresso.stg.linkedin.com:12913");
-    _manager.connect();
-    TaskDriver taskDriver = new TaskDriver(_manager);
-
-    Map<String, WorkflowConfig> workflowConfigMap = taskDriver.getWorkflows();
+    Map<String, WorkflowConfig> workflowConfigMap = _driver.getWorkflows();
     Assert.assertEquals(workflowConfigMap.size(), workflowList.size());
 
-    for(Map.Entry<String, WorkflowConfig> workflow :  workflowConfigMap.entrySet()) {
+    for (Map.Entry<String, WorkflowConfig> workflow : workflowConfigMap.entrySet()) {
       WorkflowConfig workflowConfig = workflow.getValue();
       WorkflowContext workflowContext = _driver.getWorkflowContext(workflow.getKey());
       Assert.assertNotNull(workflowContext);
 
-      for(String job : workflowConfig.getJobDag().getAllNodes()) {
+      for (String job : workflowConfig.getJobDag().getAllNodes()) {
         JobConfig jobConfig = _driver.getJobConfig(job);
         JobContext jobContext = _driver.getJobContext(job);
 


[31/33] helix git commit: Add pollForJobState and pollForWorkflowState function in TaskDriver

Posted by lx...@apache.org.
Add pollForJobState and pollForWorkflowState function in TaskDriver

1. Add pollForJobState and pollForWorkflowState functions in TaskDriver
2. Add unit tests for poll completed states for those tasks.
3. Refactor all tests using new functions.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/183a26ae
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/183a26ae
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/183a26ae

Branch: refs/heads/helix-0.6.x
Commit: 183a26ae7857a7c35985b14df9742ba3f4a06566
Parents: 9a30df4
Author: Junkai Xue <jx...@linkedin.com>
Authored: Tue May 10 11:44:25 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:21:13 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/TaskDriver.java  | 123 +++++++++++++++++++
 .../helix/integration/task/TaskTestUtil.java    |  72 +----------
 .../task/TestDisableJobExternalView.java        |   2 +-
 .../helix/integration/task/TestGenericJobs.java |   2 +-
 .../task/TestIndependentTaskRebalancer.java     |  22 ++--
 .../task/TestJobAndWorkflowType.java            |   2 +-
 .../task/TestJobFailureDependence.java          |  18 +--
 .../integration/task/TestRecurringJobQueue.java |  30 ++---
 .../integration/task/TestRetrieveWorkflows.java |   2 +-
 .../task/TestRunJobsWithMissingTarget.java      |  12 +-
 .../task/TestTaskConditionalRetry.java          |   2 +-
 .../task/TestTaskErrorReporting.java            |   2 +-
 .../integration/task/TestTaskRebalancer.java    |  18 +--
 .../task/TestTaskRebalancerFailover.java        |   6 +-
 .../task/TestTaskRebalancerRetryLimit.java      |   2 +-
 .../task/TestTaskRebalancerStopResume.java      |  61 +++++----
 .../integration/task/TestTaskRetryDelay.java    |   4 +-
 .../task/TestTaskWithInstanceDisabled.java      |   2 +-
 .../integration/task/TestUpdateWorkflow.java    |   8 +-
 .../task/TestWorkflowAndJobPoll.java            |  62 ++++++++++
 .../task/TestWorkflowJobDependency.java         |   2 +-
 21 files changed, 283 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index b3a0364..e62d15c 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -84,6 +85,10 @@ public class TaskDriver {
   /** Field for specifying a workflow file when starting a job */
   private static final String WORKFLOW_FILE_OPTION = "file";
 
+  /** Default time out for monitoring workflow or job state */
+  private final static int _defaultTimeout = 2 * 60 * 1000; /* 2 mins */
+
+
   private final HelixDataAccessor _accessor;
   private final ConfigAccessor _cfgAccessor;
   private final HelixPropertyStore<ZNRecord> _propertyStore;
@@ -824,6 +829,124 @@ public class TaskDriver {
     }
   }
 
+  /**
+   * This call will be blocked until either workflow reaches to one of the state specified
+   * in the arguments, or timeout happens. If timeout happens, then it will throw a HelixException
+   * Otherwise, it will return current workflow state
+   *
+   * @param workflowName The workflow to be monitored
+   * @param timeout A long integer presents the time out, in milliseconds
+   * @param targetStates Specified states that user would like to stop monitoring
+   * @return A TaskState, which is current workflow state
+   * @throws InterruptedException
+   */
+  public TaskState pollForWorkflowState(String workflowName, long timeout,
+      TaskState... targetStates) throws InterruptedException {
+    // Wait for completion.
+    long st = System.currentTimeMillis();
+    WorkflowContext ctx;
+    Set<TaskState> allowedStates = new HashSet<TaskState>(Arrays.asList(targetStates));
+
+    long timeToSleep = timeout > 100L ? 100L : timeout;
+    do {
+      Thread.sleep(timeToSleep);
+      ctx = getWorkflowContext(workflowName);
+    } while ((ctx == null || ctx.getWorkflowState() == null || !allowedStates
+        .contains(ctx.getWorkflowState())) && System.currentTimeMillis() < st + timeout);
+
+    if (ctx == null || !allowedStates.contains(ctx.getWorkflowState())) {
+      throw new HelixException(String
+          .format("Workflow \"%s\" context is empty or not in states: \"%s\"", workflowName,
+              targetStates));
+    }
+
+    return ctx.getWorkflowState();
+  }
+
+  /**
+   * This is a wrapper function that set default time out for monitoring workflow in 2 MINUTES.
+   * If timeout happens, then it will throw a HelixException, Otherwise, it will return
+   * current job state.
+   *
+   * @param workflowName The workflow to be monitored
+   * @param targetStates Specified states that user would like to stop monitoring
+   * @return A TaskState, which is current workflow state
+   * @throws InterruptedException
+   */
+  public TaskState pollForWorkflowState(String workflowName, TaskState... targetStates)
+      throws InterruptedException {
+    return pollForWorkflowState(workflowName, _defaultTimeout, targetStates);
+  }
+
+  /**
+   * This call will be blocked until either specified job reaches to one of the state
+   * in the arguments, or timeout happens. If timeout happens, then it will throw a HelixException
+   * Otherwise, it will return current job state
+   *
+   * @param workflowName The workflow that contains the job to monitor
+   * @param jobName The specified job to monitor
+   * @param timeout A long integer presents the time out, in milliseconds
+   * @param states Specified states that user would like to stop monitoring
+   * @return A TaskState, which is current job state
+   * @throws Exception
+   */
+  public TaskState pollForJobState(String workflowName, String jobName, long timeout,
+      TaskState... states) throws InterruptedException {
+    // Get workflow config
+    WorkflowConfig workflowConfig = getWorkflowConfig(workflowName);
+
+    if (workflowConfig == null) {
+      throw new HelixException(String.format("Workflow \"%s\" does not exists!", workflowName));
+    }
+
+    long timeToSleep = timeout > 100L ? 100L : timeout;
+
+    WorkflowContext ctx;
+    if (workflowConfig.isRecurring()) {
+      // if it's recurring, need to reconstruct workflow and job name
+      do {
+        Thread.sleep(timeToSleep);
+        ctx = getWorkflowContext(workflowName);
+      } while ((ctx == null || ctx.getLastScheduledSingleWorkflow() == null));
+
+      jobName = jobName.substring(workflowName.length() + 1);
+      workflowName = ctx.getLastScheduledSingleWorkflow();
+      jobName = TaskUtil.getNamespacedJobName(workflowName, jobName);
+    }
+
+    Set<TaskState> allowedStates = new HashSet<TaskState>(Arrays.asList(states));
+    // Wait for state
+    long st = System.currentTimeMillis();
+    do {
+      Thread.sleep(timeToSleep);
+      ctx = getWorkflowContext(workflowName);
+    } while ((ctx == null || ctx.getJobState(jobName) == null || !allowedStates
+        .contains(ctx.getJobState(jobName))) && System.currentTimeMillis() < st + timeout);
+
+    if (ctx == null || !allowedStates.contains(ctx.getJobState(jobName))) {
+      throw new HelixException(
+          String.format("Job \"%s\" context is null or not in states: \"%s\"", jobName, states));
+    }
+
+    return ctx.getJobState(jobName);
+  }
+
+  /**
+   * This is a wrapper function for monitoring job state with default timeout 2 MINUTES.
+   * If timeout happens, then it will throw a HelixException, Otherwise, it will return
+   * current job state
+   *
+   * @param workflowName The workflow that contains the job to monitor
+   * @param jobName The specified job to monitor
+   * @param states Specified states that user would like to stop monitoring
+   * @return A TaskState, which is current job state
+   * @throws Exception
+   */
+  public TaskState pollForJobState(String workflowName, String jobName, TaskState... states)
+      throws InterruptedException {
+    return pollForJobState(workflowName, jobName, _defaultTimeout, states);
+  }
+
   /** Constructs options set for all basic control messages */
   private static Options constructOptions() {
     Options options = new Options();

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
index 3e5385c..2f8fa60 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestUtil.java
@@ -18,7 +18,7 @@ package org.apache.helix.integration.task;
  * specific language governing permissions and limitations
  * under the License.
  */
-import java.util.Arrays;
+
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Date;
@@ -45,76 +45,6 @@ import org.testng.Assert;
 public class TaskTestUtil {
   private final static int _default_timeout = 2 * 60 * 1000; /* 2 mins */
 
-  /**
-   * Polls {@link org.apache.helix.task.JobContext} for given task resource until a timeout is
-   * reached.
-   * If the task has not reached target state by then, an error is thrown
-   *
-   * @param workflowResource Resource to poll for completeness
-   * @throws InterruptedException
-   */
-  public static void pollForWorkflowState(TaskDriver driver, String workflowResource,
-      TaskState... targetStates) throws InterruptedException {
-    // Wait for completion.
-    long st = System.currentTimeMillis();
-    WorkflowContext ctx;
-    Set<TaskState> allowedStates = new HashSet<TaskState>(Arrays.asList(targetStates));
-    do {
-      Thread.sleep(100);
-      ctx = driver.getWorkflowContext(workflowResource);
-    } while ((ctx == null || ctx.getWorkflowState() == null || !allowedStates
-        .contains(ctx.getWorkflowState())) && System.currentTimeMillis() < st + _default_timeout);
-
-    Assert.assertNotNull(ctx);
-    TaskState workflowState = ctx.getWorkflowState();
-    Assert.assertTrue(allowedStates.contains(workflowState),
-        "expect workflow states: " + allowedStates + " actual workflow state: " + workflowState);
-  }
-
-  /**
-   * poll for job until it is at either state in targetStates.
-   * @param driver
-   * @param workflowResource
-   * @param jobName
-   * @param targetStates
-   * @throws InterruptedException
-   */
-  public static void pollForJobState(TaskDriver driver, String workflowResource, String jobName,
-      TaskState... targetStates) throws InterruptedException {
-    // Get workflow config
-    WorkflowConfig wfCfg = driver.getWorkflowConfig(workflowResource);
-    Assert.assertNotNull(wfCfg);
-    WorkflowContext ctx;
-    if (wfCfg.isRecurring()) {
-      // if it's recurring, need to reconstruct workflow and job name
-      do {
-        Thread.sleep(100);
-        ctx = driver.getWorkflowContext(workflowResource);
-      } while ((ctx == null || ctx.getLastScheduledSingleWorkflow() == null));
-      Assert.assertNotNull(ctx);
-      Assert.assertNotNull(ctx.getLastScheduledSingleWorkflow());
-      jobName = jobName.substring(workflowResource.length() + 1);
-      workflowResource = ctx.getLastScheduledSingleWorkflow();
-      jobName = String.format("%s_%s", workflowResource, jobName);
-    }
-
-    Set<TaskState> allowedStates = new HashSet<TaskState>(Arrays.asList(targetStates));
-    // Wait for state
-    long st = System.currentTimeMillis();
-    do {
-      Thread.sleep(100);
-      ctx = driver.getWorkflowContext(workflowResource);
-    }
-    while ((ctx == null || ctx.getJobState(jobName) == null || !allowedStates.contains(
-        ctx.getJobState(jobName)))
-        && System.currentTimeMillis() < st + _default_timeout);
-    Assert.assertNotNull(ctx, "Empty job context");
-    TaskState jobState = ctx.getJobState(jobName);
-    Assert.assertTrue(allowedStates.contains(jobState),
-        "expect job " + jobName + " is in states: " + allowedStates + " actual job state: "
-            + jobState + " all other job states in the workflow: " + ctx.getJobStates().entrySet());
-  }
-
   public static void pollForEmptyJobState(final TaskDriver driver, final String workflowName,
       final String jobName) throws Exception {
     final String namespacedJobName = String.format("%s_%s", workflowName, jobName);

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
index 4563e70..061114f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestDisableJobExternalView.java
@@ -72,7 +72,7 @@ public class TestDisableJobExternalView extends TaskTestBase {
 
     // ensure all jobs are completed
     String namedSpaceJob3 = String.format("%s_%s", queueName, "job3");
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob3, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namedSpaceJob3, TaskState.COMPLETED);
 
     Set<String> seenExternalViews = externviewChecker.getSeenExternalViews();
     String namedSpaceJob1 = String.format("%s_%s", queueName, "job1");

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
index 426bade..f376b85 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
@@ -65,7 +65,7 @@ public class TestGenericJobs extends TaskTestBase {
 
     String namedSpaceJob =
         String.format("%s_%s", queueName, currentJobNames.get(currentJobNames.size() - 1));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namedSpaceJob, TaskState.COMPLETED);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
index 0e598c1..49b4bf4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
@@ -149,8 +149,9 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
+
 
     // Ensure that each class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -176,8 +177,8 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
 
     // Ensure that each class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -205,8 +206,8 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
 
     // Ensure that each class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -233,8 +234,8 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
 
     // Ensure that the class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -268,7 +269,8 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
 
     // Ensure that the class was invoked
     Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
@@ -300,7 +302,7 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
     _driver.start(workflowBuilder.build());
 
     // Ensure completion
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
 
     // Ensure a single retry happened
     JobContext jobCtx = _driver.getJobContext(jobName + "_" + jobName);

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
index 0b02085..6e74bcf 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobAndWorkflowType.java
@@ -48,7 +48,7 @@ public class TestJobAndWorkflowType extends TaskTestBase {
     // Start workflow
     _driver.start(builder.build());
 
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
     String fetchedJobType =
         _driver.getJobConfig(String.format("%s_%s", jobName, jobName)).getJobType();
     String fetchedWorkflowType =

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
index d4f6dbb..254a35d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
@@ -66,7 +66,7 @@ public class TestJobFailureDependence extends TaskTestBase {
     // all jobs after failed job should fail too.
     for (int i = 2; i < _numDbs; i++) {
       String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.FAILED);
+      _driver.pollForJobState(queueName, namedSpaceJob, TaskState.FAILED);
     }
   }
 
@@ -92,8 +92,8 @@ public class TestJobFailureDependence extends TaskTestBase {
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(2));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);
+    _driver.pollForWorkflowState(queueName, TaskState.FAILED);
   }
 
   @Test
@@ -117,12 +117,12 @@ public class TestJobFailureDependence extends TaskTestBase {
     _driver.start(queueBuilder.build());
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
     String namedSpaceJob2 = String.format("%s_%s", queueName, currentJobNames.get(2));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob2, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob2, TaskState.FAILED);
 
     // all jobs after failed job should complete.
     for (int i = 3; i < _numDbs; i++) {
       String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.COMPLETED);
+      _driver.pollForJobState(queueName, namedSpaceJob, TaskState.COMPLETED);
     }
   }
 
@@ -148,10 +148,10 @@ public class TestJobFailureDependence extends TaskTestBase {
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);
     String lastJob =
         String.format("%s_%s", queueName, currentJobNames.get(currentJobNames.size() - 1));
-    TaskTestUtil.pollForJobState(_driver, queueName, lastJob, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, lastJob, TaskState.COMPLETED);
 
     _driver.flushQueue(queueName);
 
@@ -174,8 +174,8 @@ public class TestJobFailureDependence extends TaskTestBase {
     _driver.resume(queueName);
 
     namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);
+    _driver.pollForWorkflowState(queueName, TaskState.FAILED);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index 65ec458..4d4d96a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -71,8 +71,7 @@ public class TestRecurringJobQueue extends TaskTestBase {
     // ensure job 1 is started before stop it
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
     String namedSpaceJob1 = String.format("%s_%s", scheduledQueue, currentJobNames.get(0));
-    TaskTestUtil
-        .pollForJobState(_driver, scheduledQueue, namedSpaceJob1, TaskState.IN_PROGRESS);
+    _driver.pollForJobState(scheduledQueue, namedSpaceJob1, TaskState.IN_PROGRESS);
 
     _driver.stop(queueName);
     _driver.delete(queueName);
@@ -100,13 +99,11 @@ public class TestRecurringJobQueue extends TaskTestBase {
     // ensure jobs are started and completed
     scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
     namedSpaceJob1 = String.format("%s_%s", scheduledQueue, currentJobNames.get(0));
-    TaskTestUtil
-        .pollForJobState(_driver, scheduledQueue, namedSpaceJob1, TaskState.COMPLETED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceJob1, TaskState.COMPLETED);
 
     scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
     String namedSpaceJob2 = String.format("%s_%s", scheduledQueue, currentJobNames.get(1));
-    TaskTestUtil
-        .pollForJobState(_driver, scheduledQueue, namedSpaceJob2, TaskState.COMPLETED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceJob2, TaskState.COMPLETED);
   }
 
   @Test
@@ -140,15 +137,14 @@ public class TestRecurringJobQueue extends TaskTestBase {
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", scheduledQueue, deletedJob1);
-    TaskTestUtil
-        .pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS,
-            TaskState.COMPLETED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS,
+        TaskState.COMPLETED);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + scheduledQueue);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.STOPPED);
 
     // delete the in-progress job (job 1) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob1);
@@ -159,16 +155,16 @@ public class TestRecurringJobQueue extends TaskTestBase {
     _driver.resume(queueName);
 
     // ensure job 2 is started
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
+    _driver.pollForJobState(scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.IN_PROGRESS,
         TaskState.COMPLETED);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
+    _driver.pollForJobState(scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
     String deletedJob2 = currentJobNames.get(2);
@@ -189,7 +185,7 @@ public class TestRecurringJobQueue extends TaskTestBase {
     long preJobFinish = 0;
     for (int i = 0; i < currentJobNames.size(); i++) {
       String namedSpaceJobName = String.format("%s_%s", scheduledQueue, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
+      _driver.pollForJobState(scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
 
       JobContext jobContext = _driver.getJobContext(namedSpaceJobName);
       long jobStart = jobContext.getStartTime();
@@ -238,7 +234,7 @@ public class TestRecurringJobQueue extends TaskTestBase {
 
     // ensure all jobs are finished
     String namedSpaceJob = String.format("%s_%s", scheduledQueue, currentLastJob);
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
 
     // enqueue the last job
     LOG.info("Enqueuing job: " + jobNames.get(JOB_COUNTS - 1));
@@ -286,7 +282,7 @@ public class TestRecurringJobQueue extends TaskTestBase {
 
     // ensure current schedule is started
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.COMPLETED);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
index ddd6c85..45fb278 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRetrieveWorkflows.java
@@ -43,7 +43,7 @@ public class TestRetrieveWorkflows extends TaskTestBase {
     }
 
     for (Workflow workflow : workflowList) {
-      TaskTestUtil.pollForWorkflowState(_driver, workflow.getName(), TaskState.COMPLETED);
+      _driver.pollForWorkflowState(workflow.getName(), TaskState.COMPLETED);
     }
 
     Map<String, WorkflowConfig> workflowConfigMap = _driver.getWorkflows();

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index 5a07942..879bfea 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -64,8 +64,8 @@ public class TestRunJobsWithMissingTarget extends TaskTestBase {
     _driver.start(queueBuilder.build());
 
     String namedSpaceJob = String.format("%s_%s", queueName, currentJobNames.get(1));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.FAILED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob, TaskState.FAILED);
+    _driver.pollForWorkflowState(queueName, TaskState.FAILED);
 
     _driver.delete(queueName);
   }
@@ -91,10 +91,10 @@ public class TestRunJobsWithMissingTarget extends TaskTestBase {
     _driver.start(queueBuilder.build());
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);
     String lastJob =
         String.format("%s_%s", queueName, currentJobNames.get(currentJobNames.size() - 1));
-    TaskTestUtil.pollForJobState(_driver, queueName, lastJob, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, lastJob, TaskState.COMPLETED);
 
     _driver.delete(queueName);
   }
@@ -121,8 +121,8 @@ public class TestRunJobsWithMissingTarget extends TaskTestBase {
     _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(0));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(0));
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob1, TaskState.FAILED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.FAILED);
+    _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);
+    _driver.pollForWorkflowState(queueName, TaskState.FAILED);
 
     _driver.delete(queueName);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
index 5fa370d..b742eae 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
@@ -79,7 +79,7 @@ public class TestTaskConditionalRetry extends TaskTestBase {
     _driver.start(flow);
 
     // Wait until the job completes.
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     for (int i = 0; i < num_tasks; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
index 906dcff..80176da 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskErrorReporting.java
@@ -90,7 +90,7 @@ public class TestTaskErrorReporting extends TaskTestBase {
     _driver.start(flow);
 
     // Wait until the job completes.
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     for (int i = 0; i < num_tasks; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index f5a3441..4725c20 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -72,7 +72,7 @@ public class TestTaskRebalancer extends TaskTestBase {
             .setExpiry(expiry).build();
 
     _driver.start(flow);
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
 
     // Running workflow should have config and context viewable through accessor
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
@@ -86,7 +86,7 @@ public class TestTaskRebalancer extends TaskTestBase {
     Assert.assertNotSame(accessor.getProperty(workflowCfgKey), null);
 
     // Wait for job to finish and expire
-    TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
     Thread.sleep(expiry + 100);
 
     // Ensure workflow config and context were cleaned up by now
@@ -114,7 +114,7 @@ public class TestTaskRebalancer extends TaskTestBase {
     _driver.start(flow);
 
     // Wait for job completion
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     // Ensure all partitions are completed individually
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
@@ -141,7 +141,7 @@ public class TestTaskRebalancer extends TaskTestBase {
     _driver.start(flow);
 
     // wait for job completeness/timeout
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     // see if resulting context completed successfully for our partition set
     String namespacedName = TaskUtil.getNamespacedJobName(jobResource);
@@ -166,11 +166,11 @@ public class TestTaskRebalancer extends TaskTestBase {
     new TaskDriver(_manager).start(flow);
 
     // Wait until the workflow completes
-    TaskTestUtil.pollForWorkflowState(_driver, workflowName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(workflowName, TaskState.COMPLETED);
 
     // Assert completion for all tasks within two minutes
     for (String task : flow.getJobConfigs().keySet()) {
-      TaskTestUtil.pollForJobState(_driver, workflowName, task, TaskState.COMPLETED);
+      _driver.pollForJobState(workflowName, task, TaskState.COMPLETED);
     }
   }
 
@@ -186,7 +186,7 @@ public class TestTaskRebalancer extends TaskTestBase {
     _driver.start(flow);
 
     // Wait until the job reports failure.
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.FAILED);
+    _driver.pollForWorkflowState(jobResource, TaskState.FAILED);
 
     // Check that all partitions timed out up to maxAttempts
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
@@ -224,8 +224,8 @@ public class TestTaskRebalancer extends TaskTestBase {
     // Ensure successful completion
     String namespacedJob1 = queueName + "_masterJob";
     String namespacedJob2 = queueName + "_slaveJob";
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob1, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob2, TaskState.COMPLETED);
     JobContext masterJobContext = _driver.getJobContext(namespacedJob1);
     JobContext slaveJobContext = _driver.getJobContext(namespacedJob2);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
index 9d98ba9..0f87e1d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerFailover.java
@@ -60,7 +60,7 @@ public class TestTaskRebalancerFailover extends TaskTestBase {
 
     // check all tasks completed on MASTER
     String namespacedJob1 = String.format("%s_%s", queueName, job1Name);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob1, TaskState.COMPLETED);
 
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
@@ -87,9 +87,9 @@ public class TestTaskRebalancerFailover extends TaskTestBase {
     LOG.info("Enqueuing job: " + job2Name);
     _driver.enqueueJob(queueName, job2Name, job);
 
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.IN_PROGRESS);
+    _driver.pollForJobState(queueName, namespacedJob2, TaskState.IN_PROGRESS);
     _participants[0].syncStop();
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob2, TaskState.COMPLETED);
 
     // tasks previously assigned to localhost_12918 should be re-scheduled on new master
     ctx = _driver.getJobContext(namespacedJob2);

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
index d677920..7f3d9d3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerRetryLimit.java
@@ -53,7 +53,7 @@ public class TestTaskRebalancerRetryLimit extends TaskTestBase {
     _driver.start(flow);
 
     // Wait until the job completes.
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     for (int i = 0; i < _numParitions; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
index e92a129..8e1f47d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
@@ -66,15 +66,15 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
 
     LOG.info("Starting flow " + flow.getName());
     _driver.start(flow);
-    TaskTestUtil.pollForWorkflowState(_driver, JOB_RESOURCE, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(JOB_RESOURCE, TaskState.IN_PROGRESS);
 
     LOG.info("Pausing job");
     _driver.stop(JOB_RESOURCE);
-    TaskTestUtil.pollForWorkflowState(_driver, JOB_RESOURCE, TaskState.STOPPED);
+    _driver.pollForWorkflowState(JOB_RESOURCE, TaskState.STOPPED);
 
     LOG.info("Resuming job");
     _driver.resume(JOB_RESOURCE);
-    TaskTestUtil.pollForWorkflowState(_driver, JOB_RESOURCE, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(JOB_RESOURCE, TaskState.COMPLETED);
   }
 
   @Test
@@ -84,15 +84,15 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
 
     LOG.info("Starting flow " + workflow);
     _driver.start(flow);
-    TaskTestUtil.pollForWorkflowState(_driver, workflow, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(workflow, TaskState.IN_PROGRESS);
 
     LOG.info("Pausing workflow");
     _driver.stop(workflow);
-    TaskTestUtil.pollForWorkflowState(_driver, workflow, TaskState.STOPPED);
+    _driver.pollForWorkflowState(workflow, TaskState.STOPPED);
 
     LOG.info("Resuming workflow");
     _driver.resume(workflow);
-    TaskTestUtil.pollForWorkflowState(_driver, workflow, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(workflow, TaskState.COMPLETED);
   }
 
   @Test
@@ -122,13 +122,13 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     _driver.enqueueJob(queueName, job2Name, job2);
 
     String namespacedJob1 = String.format("%s_%s", queueName,  job1Name);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.IN_PROGRESS);
+    _driver.pollForJobState(queueName, namespacedJob1, TaskState.IN_PROGRESS);
 
     // stop job1
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.STOPPED);
+    _driver.pollForJobState(queueName, namespacedJob1, TaskState.STOPPED);
+    _driver.pollForWorkflowState(queueName, TaskState.STOPPED);
 
     // Ensure job2 is not started
     TimeUnit.MILLISECONDS.sleep(200);
@@ -139,8 +139,8 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     _driver.resume(queueName);
 
     // Ensure successful completion
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob1, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob2, TaskState.COMPLETED);
     JobContext masterJobContext = _driver.getJobContext(namespacedJob1);
     JobContext slaveJobContext = _driver.getJobContext(namespacedJob2);
 
@@ -188,13 +188,13 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", queueName, deletedJob1);
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
+    _driver.pollForJobState(queueName, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceDeletedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.STOPPED);
+    _driver.pollForJobState(queueName, namedSpaceDeletedJob1, TaskState.STOPPED);
+    _driver.pollForWorkflowState(queueName, TaskState.STOPPED);
 
     // delete the in-progress job (job 1) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob1);
@@ -204,15 +204,15 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     _driver.resume(queueName);
 
     // ensure job 2 is started
-    TaskTestUtil
-        .pollForJobState(_driver, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.IN_PROGRESS);
+    _driver.pollForJobState(queueName, String.format("%s_%s", queueName, currentJobNames.get(1)),
+        TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil
-        .pollForJobState(_driver, queueName, String.format("%s_%s", queueName, currentJobNames.get(1)), TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, queueName, TaskState.STOPPED);
+    _driver.pollForJobState(queueName, String.format("%s_%s", queueName, currentJobNames.get(1)),
+        TaskState.STOPPED);
+    _driver.pollForWorkflowState(queueName, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
     String deletedJob2 = currentJobNames.get(2);
@@ -241,7 +241,7 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     long preJobFinish = 0;
     for (int i = 0; i < currentJobNames.size(); i++) {
       String namedSpaceJobName = String.format("%s_%s", queueName, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJobName, TaskState.COMPLETED);
+      _driver.pollForJobState(queueName, namedSpaceJobName, TaskState.COMPLETED);
 
       JobContext jobContext = _driver.getJobContext(namedSpaceJobName);
       long jobStart = jobContext.getStartTime();
@@ -295,14 +295,13 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     // ensure job 1 is started before deleting it
     String deletedJob1 = currentJobNames.get(0);
     String namedSpaceDeletedJob1 = String.format("%s_%s", scheduledQueue, deletedJob1);
-    TaskTestUtil
-        .pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
+    _driver.pollForJobState(scheduledQueue, namedSpaceDeletedJob1, TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + scheduledQueue);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceDeletedJob1, TaskState.STOPPED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.STOPPED);
 
     // delete the in-progress job (job 1) and verify it being deleted
     _driver.deleteJob(queueName, deletedJob1);
@@ -313,15 +312,15 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     _driver.resume(queueName);
 
     // ensure job 2 is started
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
+    _driver.pollForJobState(scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.IN_PROGRESS);
 
     // stop the queue
     LOG.info("Pausing job-queue: " + queueName);
     _driver.stop(queueName);
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue,
+    _driver.pollForJobState(scheduledQueue,
         String.format("%s_%s", scheduledQueue, currentJobNames.get(1)), TaskState.STOPPED);
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.STOPPED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.STOPPED);
 
     // Ensure job 3 is not started before deleting it
     String deletedJob2 = currentJobNames.get(2);
@@ -342,7 +341,7 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     long preJobFinish = 0;
     for (int i = 0; i < currentJobNames.size(); i++) {
       String namedSpaceJobName = String.format("%s_%s", scheduledQueue, currentJobNames.get(i));
-      TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
+      _driver.pollForJobState(scheduledQueue, namedSpaceJobName, TaskState.COMPLETED);
 
       JobContext jobContext = _driver.getJobContext(namedSpaceJobName);
       long jobStart = jobContext.getStartTime();
@@ -390,7 +389,7 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
 
     // ensure all jobs are finished
     String namedSpaceJob = String.format("%s_%s", scheduledQueue, currentLastJob);
-    TaskTestUtil.pollForJobState(_driver, scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
+    _driver.pollForJobState(scheduledQueue, namedSpaceJob, TaskState.COMPLETED);
 
     // enqueue the last job
     LOG.info("Enqueuing job: " + jobNames.get(JOB_COUNTS - 1));
@@ -433,10 +432,10 @@ public class TestTaskRebalancerStopResume extends TaskTestBase {
     _driver.enqueueJob(queueName, job2Name, job2);
 
     String namespacedJob1 = String.format("%s_%s", queueName,  job1Name);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob1, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob1, TaskState.COMPLETED);
 
     String namespacedJob2 = String.format("%s_%s", queueName,  job2Name);
-    TaskTestUtil.pollForJobState(_driver, queueName, namespacedJob2, TaskState.COMPLETED);
+    _driver.pollForJobState(queueName, namespacedJob2, TaskState.COMPLETED);
 
     // Stop queue
     _driver.stop(queueName);

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
index 7d8ebff..47624e4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRetryDelay.java
@@ -49,7 +49,7 @@ public class TestTaskRetryDelay extends TaskTestBase {
     _driver.start(flow);
 
     // Wait until the job completes.
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     long startTime = _driver.getWorkflowContext(jobResource).getStartTime();
     long finishedTime = _driver.getWorkflowContext(jobResource).getFinishTime();
@@ -70,7 +70,7 @@ public class TestTaskRetryDelay extends TaskTestBase {
     _driver.start(flow);
 
     // Wait until the job completes.
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
 
     long startTime = _driver.getWorkflowContext(jobResource).getStartTime();
     long finishedTime = _driver.getWorkflowContext(jobResource).getFinishTime();

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
index 84e5168..919dc99 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
@@ -51,7 +51,7 @@ public class TestTaskWithInstanceDisabled extends TaskTestBase {
         WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
     _driver.start(flow);
 
-    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(jobResource, TaskState.COMPLETED);
     JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
     Assert.assertEquals(ctx.getAssignedParticipant(0), PARTICIPANT_PREFIX + "_" + (_startPort + 1));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
index b43c49e..73e6355 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUpdateWorkflow.java
@@ -63,12 +63,12 @@ public class TestUpdateWorkflow extends TaskTestBase {
 
     // ensure current schedule is started
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.IN_PROGRESS);
 
     _driver.updateWorkflow(queueName, configBuilder.build());
 
     // ensure current schedule is completed
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.COMPLETED);
 
     Thread.sleep(1000);
 
@@ -98,7 +98,7 @@ public class TestUpdateWorkflow extends TaskTestBase {
 
     // ensure current schedule is started
     String scheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.IN_PROGRESS);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.IN_PROGRESS);
 
     _driver.stop(queueName);
 
@@ -122,7 +122,7 @@ public class TestUpdateWorkflow extends TaskTestBase {
     _driver.resume(queueName);
 
     // ensure current schedule is completed
-    TaskTestUtil.pollForWorkflowState(_driver, scheduledQueue, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(scheduledQueue, TaskState.COMPLETED);
 
     Thread.sleep(1000);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowAndJobPoll.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowAndJobPoll.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowAndJobPoll.java
new file mode 100644
index 0000000..91b65a9
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowAndJobPoll.java
@@ -0,0 +1,62 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestWorkflowAndJobPoll extends TaskTestBase {
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numDbs = 1;
+    _numNodes = 1;
+    _numParitions = 1;
+    _numReplicas = 1;
+    super.beforeClass();
+  }
+
+  @Test public void testWorkflowPoll() throws InterruptedException {
+    String jobResource = TestHelper.getTestMethodName();
+    Workflow.Builder builder =
+        WorkflowGenerator.generateDefaultSingleJobWorkflowBuilder(jobResource);
+    _driver.start(builder.build());
+
+    TaskState polledState =
+        _driver.pollForWorkflowState(jobResource, 4000L, TaskState.COMPLETED, TaskState.FAILED);
+    Assert.assertEquals(TaskState.COMPLETED, polledState);
+  }
+
+  @Test public void testJobPoll() throws InterruptedException {
+    String jobResource = TestHelper.getTestMethodName();
+    Workflow.Builder builder =
+        WorkflowGenerator.generateDefaultSingleJobWorkflowBuilder(jobResource);
+    _driver.start(builder.build());
+
+    TaskState polledState = _driver
+        .pollForJobState(jobResource, String.format("%s_%s", jobResource, jobResource), 4000L,
+            TaskState.COMPLETED, TaskState.FAILED);
+    Assert.assertEquals(TaskState.COMPLETED, polledState);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/183a26ae/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
index 1ced40b..6cae0e5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestWorkflowJobDependency.java
@@ -65,7 +65,7 @@ public class TestWorkflowJobDependency extends TaskTestBase {
     _driver.start(workflow);
 
     // Wait until the workflow completes
-    TaskTestUtil.pollForWorkflowState(_driver, workflowName, TaskState.COMPLETED);
+    _driver.pollForWorkflowState(workflowName, TaskState.COMPLETED);
     WorkflowContext workflowContext = _driver.getWorkflowContext(workflowName);
     long startTime = workflowContext.getStartTime();
     long finishTime = workflowContext.getFinishTime();


[14/33] helix git commit: Add Builder class for TaskConfig, and add unit test for testing generic jobs.

Posted by lx...@apache.org.
Add Builder class for TaskConfig, and add unit test for testing generic jobs.


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

Branch: refs/heads/helix-0.6.x
Commit: b6b89de5cf00c1d1d1cba2cd09fcd25054b2247d
Parents: 57bfc4a
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Mar 30 10:08:42 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:56:40 2016 -0700

----------------------------------------------------------------------
 .../helix/task/DeprecatedTaskRebalancer.java    |   2 +-
 .../java/org/apache/helix/task/TaskConfig.java  | 165 +++++++++++++------
 .../org/apache/helix/task/TaskStateModel.java   |   2 +-
 .../java/org/apache/helix/task/TaskUtil.java    |   2 +-
 .../java/org/apache/helix/task/Workflow.java    |   2 +-
 .../apache/helix/task/WorkflowRebalancer.java   |   2 +-
 .../helix/integration/task/TestGenericJobs.java | 162 ++++++++++++++++++
 7 files changed, 283 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
index 855312b..fbc4483 100644
--- a/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
@@ -708,7 +708,7 @@ public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCal
         Map<String, Map<String, String>> rawTaskConfigMap = jobConfig.getRecord().getMapFields();
         List<TaskConfig> taskConfigs = Lists.newLinkedList();
         for (Map<String, String> rawTaskConfig : rawTaskConfigMap.values()) {
-          TaskConfig taskConfig = TaskConfig.from(rawTaskConfig);
+          TaskConfig taskConfig = TaskConfig.Builder.from(rawTaskConfig);
           taskConfigs.add(taskConfig);
         }
         builder.addTaskConfigs(job, taskConfigs);

http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java b/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java
index 3e24725..b990f99 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java
@@ -33,7 +33,7 @@ import com.google.common.collect.Maps;
  * Configuration for an individual task to be run as part of a job.
  */
 public class TaskConfig {
-  private enum TaskConfigFields {
+  private enum TaskConfigProperty {
     TASK_ID,
     TASK_COMMAND,
     TASK_SUCCESS_OPTIONAL,
@@ -46,12 +46,13 @@ public class TaskConfig {
 
   /**
    * Instantiate the task config
-   * @param command the command to invoke for the task
-   * @param configMap configuration to be passed as part of the invocation
+   *
+   * @param command         the command to invoke for the task
+   * @param configMap       configuration to be passed as part of the invocation
    * @param successOptional true if this task need not pass for the job to succeed, false
-   *          otherwise
-   * @param id existing task ID
-   * @param target target partition for a task
+   *                        otherwise
+   * @param id              existing task ID
+   * @param target          target partition for a task
    */
   public TaskConfig(String command, Map<String, String> configMap, boolean successOptional,
       String id, String target) {
@@ -62,23 +63,24 @@ public class TaskConfig {
       id = UUID.randomUUID().toString();
     }
     if (command != null) {
-      configMap.put(TaskConfigFields.TASK_COMMAND.toString(), command);
+      configMap.put(TaskConfigProperty.TASK_COMMAND.name(), command);
     }
-    configMap.put(TaskConfigFields.TASK_SUCCESS_OPTIONAL.toString(),
-        Boolean.toString(successOptional));
-    configMap.put(TaskConfigFields.TASK_ID.toString(), id);
+    configMap
+        .put(TaskConfigProperty.TASK_SUCCESS_OPTIONAL.name(), Boolean.toString(successOptional));
+    configMap.put(TaskConfigProperty.TASK_ID.name(), id);
     if (target != null) {
-      configMap.put(TaskConfigFields.TASK_TARGET_PARTITION.toString(), target);
+      configMap.put(TaskConfigProperty.TASK_TARGET_PARTITION.name(), target);
     }
     _configMap = configMap;
   }
 
   /**
    * Instantiate the task config
-   * @param command the command to invoke for the task
-   * @param configMap configuration to be passed as part of the invocation
+   *
+   * @param command         the command to invoke for the task
+   * @param configMap       configuration to be passed as part of the invocation
    * @param successOptional true if this task need not pass for the job to succeed, false
-   *          otherwise
+   *                        otherwise
    */
   public TaskConfig(String command, Map<String, String> configMap, boolean successOptional) {
     this(command, configMap, successOptional, null, null);
@@ -86,34 +88,38 @@ public class TaskConfig {
 
   /**
    * Unique identifier for this task
+   *
    * @return UUID as a string
    */
   public String getId() {
-    return _configMap.get(TaskConfigFields.TASK_ID.toString());
+    return _configMap.get(TaskConfigProperty.TASK_ID.name());
   }
 
   /**
    * Get the command to invoke for this task
+   *
    * @return string command, or null if not overridden
    */
   public String getCommand() {
-    return _configMap.get(TaskConfigFields.TASK_COMMAND.toString());
+    return _configMap.get(TaskConfigProperty.TASK_COMMAND.name());
   }
 
   /**
    * Get the target partition of this task, if any
+   *
    * @return the target partition, or null
    */
   public String getTargetPartition() {
-    return _configMap.get(TaskConfigFields.TASK_TARGET_PARTITION.toString());
+    return _configMap.get(TaskConfigProperty.TASK_TARGET_PARTITION.name());
   }
 
   /**
    * Check if this task must succeed for a job to succeed
+   *
    * @return true if success is optional, false otherwise
    */
   public boolean isSuccessOptional() {
-    String successOptionalStr = _configMap.get(TaskConfigFields.TASK_SUCCESS_OPTIONAL.toString());
+    String successOptionalStr = _configMap.get(TaskConfigProperty.TASK_SUCCESS_OPTIONAL.name());
     if (successOptionalStr == null) {
       return false;
     } else {
@@ -123,14 +129,14 @@ public class TaskConfig {
 
   /**
    * Get the configuration map for this task's command
+   *
    * @return map of configuration key to value
    */
   public Map<String, String> getConfigMap() {
     return _configMap;
   }
 
-  @Override
-  public String toString() {
+  @Override public String toString() {
     ObjectMapper mapper = new ObjectMapper();
     try {
       return mapper.writeValueAsString(this);
@@ -140,36 +146,97 @@ public class TaskConfig {
     return super.toString();
   }
 
-  /**
-   * Instantiate a typed configuration from just a target
-   * @param target the target partition
-   * @return instantiated TaskConfig
-   */
-  public static TaskConfig from(String target) {
-    return new TaskConfig(null, null, false, null, target);
-  }
+  public static class Builder {
+    private String _taskId;
+    private String _command;
+    private String _targetPartition;
+    private boolean _successOptional = false;
+    private Map<String, String> _configMap;
 
-  /**
-   * Instantiate a typed configuration from a bean
-   * @param bean plain bean describing the task
-   * @return instantiated TaskConfig
-   */
-  public static TaskConfig from(TaskBean bean) {
-    return new TaskConfig(bean.command, bean.taskConfigMap, bean.successOptional);
-  }
+    public TaskConfig build() {
+      return new TaskConfig(_command, _configMap, _successOptional, _taskId, _targetPartition);
+    }
 
-  /**
-   * Instantiate a typed configuration from a raw string map
-   * @param rawConfigMap mixed map of configuration and task metadata
-   * @return instantiated TaskConfig
-   */
-  public static TaskConfig from(Map<String, String> rawConfigMap) {
-    String taskId = rawConfigMap.get(TaskConfigFields.TASK_ID.toString());
-    String command = rawConfigMap.get(TaskConfigFields.TASK_COMMAND.toString());
-    String targetPartition = rawConfigMap.get(TaskConfigFields.TASK_TARGET_PARTITION.toString());
-    String successOptionalStr = rawConfigMap.get(TaskConfigFields.TASK_SUCCESS_OPTIONAL.toString());
-    boolean successOptional =
-        (successOptionalStr != null) ? Boolean.valueOf(successOptionalStr) : null;
-    return new TaskConfig(command, rawConfigMap, successOptional, taskId, targetPartition);
+    public String getTaskId() {
+      return _taskId;
+    }
+
+    public Builder setTaskId(String taskId) {
+      _taskId = taskId;
+      return this;
+    }
+
+    public String getCommand() {
+      return _command;
+    }
+
+    public Builder setCommand(String command) {
+      _command = command;
+      return this;
+    }
+
+    public String getTargetPartition() {
+      return _targetPartition;
+    }
+
+    public Builder setTargetPartition(String targetPartition) {
+      _targetPartition = targetPartition;
+      return this;
+    }
+
+    public boolean isSuccessOptional() {
+      return _successOptional;
+    }
+
+    public Builder setSuccessOptional(boolean successOptional) {
+      _successOptional = successOptional;
+      return this;
+    }
+
+    public Builder addConfig(String key, String value) {
+      if (_configMap == null) {
+        _configMap = Maps.newHashMap();
+      }
+      _configMap.put(key, value);
+      return this;
+    }
+
+    /**
+     * Instantiate a typed configuration from just a target
+     *
+     * @param target the target partition
+     * @return instantiated TaskConfig
+     */
+    public static TaskConfig from(String target) {
+      return new TaskConfig(null, null, false, null, target);
+    }
+
+    /**
+     * Instantiate a typed configuration from a bean
+     *
+     * @param bean plain bean describing the task
+     * @return instantiated TaskConfig
+     */
+    public static TaskConfig from(TaskBean bean) {
+      return new TaskConfig(bean.command, bean.taskConfigMap, bean.successOptional);
+    }
+
+    /**
+     * Instantiate a typed configuration from a raw string map
+     *
+     * @param rawConfigMap mixed map of configuration and task metadata
+     * @return instantiated TaskConfig
+     */
+    @Deprecated
+    public static TaskConfig from(Map<String, String> rawConfigMap) {
+      String taskId = rawConfigMap.get(TaskConfigProperty.TASK_ID.name());
+      String command = rawConfigMap.get(TaskConfigProperty.TASK_COMMAND.name());
+      String targetPartition = rawConfigMap.get(TaskConfigProperty.TASK_TARGET_PARTITION.name());
+      String successOptionalStr =
+          rawConfigMap.get(TaskConfigProperty.TASK_SUCCESS_OPTIONAL.name());
+      boolean successOptional =
+          (successOptionalStr != null) ? Boolean.valueOf(successOptionalStr) : false;
+      return new TaskConfig(command, rawConfigMap, successOptional, taskId, targetPartition);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java b/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
index 525a38b..d3ee003 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
@@ -255,7 +255,7 @@ public class TaskStateModel extends StateModel {
     // Report a target if that was used to assign the partition
     String target = ctx.getTargetForPartition(pId);
     if (taskConfig == null && target != null) {
-      taskConfig = TaskConfig.from(target);
+      taskConfig = TaskConfig.Builder.from(target);
     }
 
     // Populate a task callback context

http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
index 8745a82..44de175 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
@@ -69,7 +69,7 @@ public class TaskUtil {
         jobResourceConfig.getRecord().getMapFields();
     Map<String, TaskConfig> taskConfigMap = Maps.newHashMap();
     for (Map<String, String> rawTaskConfig : rawTaskConfigMap.values()) {
-      TaskConfig taskConfig = TaskConfig.from(rawTaskConfig);
+      TaskConfig taskConfig = TaskConfig.Builder.from(rawTaskConfig);
       taskConfigMap.put(taskConfig.getId(), taskConfig);
     }
     b.addTaskConfigMap(taskConfigMap);

http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/main/java/org/apache/helix/task/Workflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/Workflow.java b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
index f3abc2e..e077f47 100644
--- a/helix-core/src/main/java/org/apache/helix/task/Workflow.java
+++ b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
@@ -187,7 +187,7 @@ public class Workflow {
         if (job.tasks != null) {
           List<TaskConfig> taskConfigs = Lists.newArrayList();
           for (TaskBean task : job.tasks) {
-            taskConfigs.add(TaskConfig.from(task));
+            taskConfigs.add(TaskConfig.Builder.from(task));
           }
           workflowBuilder.addTaskConfigs(job.name, taskConfigs);
         }

http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
index 9d1106a..8f97cce 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
@@ -368,7 +368,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
         Map<String, Map<String, String>> rawTaskConfigMap = jobConfig.getRecord().getMapFields();
         List<TaskConfig> taskConfigs = Lists.newLinkedList();
         for (Map<String, String> rawTaskConfig : rawTaskConfigMap.values()) {
-          TaskConfig taskConfig = TaskConfig.from(rawTaskConfig);
+          TaskConfig taskConfig = TaskConfig.Builder.from(rawTaskConfig);
           taskConfigs.add(taskConfig);
         }
         jobCfgBuilder.addTaskConfigs(taskConfigs);

http://git-wip-us.apache.org/repos/asf/helix/blob/b6b89de5/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
new file mode 100644
index 0000000..d96acd9
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericJobs.java
@@ -0,0 +1,162 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.TestHelper;
+import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.Task;
+import org.apache.helix.task.TaskCallbackContext;
+import org.apache.helix.task.TaskConfig;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.TaskFactory;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskStateModelFactory;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestGenericJobs extends ZkIntegrationTestBase {
+  private static final Logger LOG = Logger.getLogger(TestGenericJobs.class);
+  private static final int num_nodes = 5;
+  private static final int START_PORT = 12918;
+  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
+  private final MockParticipantManager[] _participants = new MockParticipantManager[num_nodes];
+  private ClusterControllerManager _controller;
+  private ClusterSetup _setupTool;
+
+  private HelixManager _manager;
+  private TaskDriver _driver;
+
+  @BeforeClass public void beforeClass() throws Exception {
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+
+    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool.addCluster(CLUSTER_NAME, true);
+    for (int i = 0; i < num_nodes; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    }
+
+    Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
+    taskFactoryReg.put(MockTask.TASK_COMMAND, new TaskFactory() {
+      @Override public Task createNewTask(TaskCallbackContext context) {
+        return new MockTask(context);
+      }
+    });
+
+    // start dummy participants
+    for (int i = 0; i < num_nodes; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+
+      // Register a Task state model factory.
+      StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
+      stateMachine.registerStateModelFactory("Task",
+          new TaskStateModelFactory(_participants[i], taskFactoryReg));
+
+      _participants[i].syncStart();
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // create cluster manager
+    _manager = HelixManagerFactory
+        .getZKHelixManager(CLUSTER_NAME, "Admin", InstanceType.ADMINISTRATOR, ZK_ADDR);
+    _manager.connect();
+
+    _driver = new TaskDriver(_manager);
+
+    boolean result = ClusterStateVerifier.verifyByZkCallback(
+        new ClusterStateVerifier.MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+
+    result = ClusterStateVerifier.verifyByZkCallback(
+        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
+    Assert.assertTrue(result);
+  }
+
+  @AfterClass public void afterClass() throws Exception {
+    _manager.disconnect();
+    _controller.syncStop();
+    for (int i = 0; i < num_nodes; i++) {
+      _participants[i].syncStop();
+    }
+    _setupTool.deleteCluster(CLUSTER_NAME);
+  }
+
+  @Test public void testGenericJobs() throws Exception {
+    String queueName = TestHelper.getTestMethodName();
+
+    // Create a queue
+    LOG.info("Starting job-queue: " + queueName);
+    JobQueue.Builder queueBuilder = TaskTestUtil.buildJobQueue(queueName);
+
+    // Create and Enqueue jobs
+    int num_jobs = 4;
+    List<String> currentJobNames = new ArrayList<String>();
+    for (int i = 0; i < num_jobs; i++) {
+      JobConfig.Builder jobConfig = new JobConfig.Builder();
+
+      // create each task configs.
+      List<TaskConfig> taskConfigs = new ArrayList<TaskConfig>();
+      int num_tasks = 10;
+      for (int j = 0; j < num_tasks; j++) {
+        taskConfigs.add(
+            new TaskConfig.Builder().setTaskId("task_" + j).setCommand(MockTask.TASK_COMMAND)
+                .build());
+      }
+      jobConfig.addTaskConfigs(taskConfigs);
+
+      String jobName = "job_" + i;
+      queueBuilder.enqueueJob(jobName, jobConfig);
+      currentJobNames.add(jobName);
+    }
+
+    _driver.start(queueBuilder.build());
+
+    String namedSpaceJob =
+        String.format("%s_%s", queueName, currentJobNames.get(currentJobNames.size() - 1));
+    TaskTestUtil.pollForJobState(_driver, queueName, namedSpaceJob, TaskState.COMPLETED);
+  }
+}
+


[15/33] helix git commit: More cleanup on workflow and workflowConfig builders.

Posted by lx...@apache.org.
More cleanup on workflow and workflowConfig builders.


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

Branch: refs/heads/helix-0.6.x
Commit: 1f683b863df23f16bd893fc675f88ed8b7f3d3b8
Parents: b6b89de
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Mar 30 13:59:59 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:58:18 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobConfig.java   | 40 ++++++++++++++++
 .../java/org/apache/helix/task/Workflow.java    | 50 ++------------------
 .../org/apache/helix/task/WorkflowConfig.java   | 11 +++++
 .../apache/helix/task/WorkflowRebalancer.java   |  2 +-
 .../org/apache/helix/task/beans/JobBean.java    |  4 +-
 .../task/TestIndependentTaskRebalancer.java     | 12 ++---
 .../integration/task/WorkflowGenerator.java     |  4 +-
 7 files changed, 67 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index d423d38..4d5aa94 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -30,7 +30,10 @@ import java.util.Set;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.helix.task.beans.JobBean;
+import org.apache.helix.task.beans.TaskBean;
 
 /**
  * Provides a typed interface to job configurations.
@@ -485,6 +488,43 @@ public class JobConfig {
       }
     }
 
+    public static Builder from(JobBean jobBean) {
+      Builder b = new Builder();
+
+      b.setMaxAttemptsPerTask(jobBean.maxAttemptsPerTask)
+          .setMaxForcedReassignmentsPerTask(jobBean.maxForcedReassignmentsPerTask)
+          .setNumConcurrentTasksPerInstance(jobBean.numConcurrentTasksPerInstance)
+          .setTimeoutPerTask(jobBean.timeoutPerPartition)
+          .setFailureThreshold(jobBean.failureThreshold).setTaskRetryDelay(jobBean.taskRetryDelay)
+          .setDisableExternalView(jobBean.disableExternalView)
+          .setIgnoreDependentJobFailure(jobBean.ignoreDependentJobFailure);
+
+      if (jobBean.jobCommandConfigMap != null) {
+        b.setJobCommandConfigMap(jobBean.jobCommandConfigMap);
+      }
+      if (jobBean.command != null) {
+        b.setCommand(jobBean.command);
+      }
+      if (jobBean.targetResource != null) {
+        b.setTargetResource(jobBean.targetResource);
+      }
+      if (jobBean.targetPartitionStates != null) {
+        b.setTargetPartitionStates(new HashSet<String>(jobBean.targetPartitionStates));
+      }
+      if (jobBean.targetPartitions != null) {
+        b.setTargetPartitions(jobBean.targetPartitions);
+      }
+      if (jobBean.tasks != null) {
+        List<TaskConfig> taskConfigs = Lists.newArrayList();
+        for (TaskBean task : jobBean.tasks) {
+          taskConfigs.add(TaskConfig.Builder.from(task));
+        }
+        b.addTaskConfigs(taskConfigs);
+      }
+
+      return b;
+    }
+
     private static List<String> csvToStringList(String csv) {
       String[] vals = csv.split(",");
       return Arrays.asList(vals);

http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/main/java/org/apache/helix/task/Workflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/Workflow.java b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
index e077f47..a7060c3 100644
--- a/helix-core/src/main/java/org/apache/helix/task/Workflow.java
+++ b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
@@ -34,14 +34,10 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.task.beans.JobBean;
-import org.apache.helix.task.beans.TaskBean;
 import org.apache.helix.task.beans.WorkflowBean;
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.Constructor;
 
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
-
 /**
  * Houses a job dag and config set to fully describe a job workflow
  */
@@ -150,56 +146,18 @@ public class Workflow {
         if (job.name == null) {
           throw new IllegalArgumentException("A job must have a name.");
         }
-
+        JobConfig.Builder jobConfigBuilder = JobConfig.Builder.from(job);
+        jobConfigBuilder.setWorkflow(wf.name);
+        workflowBuilder.addJob(job.name, jobConfigBuilder);
         if (job.parents != null) {
           for (String parent : job.parents) {
             workflowBuilder.addParentChildDependency(parent, job.name);
           }
         }
-
-        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.WorkflowID.name(), wf.name);
-        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.Command.name(), job.command);
-        if (job.jobConfigMap != null) {
-          workflowBuilder.addJobCommandConfigMap(job.name, job.jobConfigMap);
-        }
-        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TargetResource.name(),
-            job.targetResource);
-        if (job.targetPartitionStates != null) {
-          workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TargetPartitionStates.name(),
-              Joiner.on(",").join(job.targetPartitionStates));
-        }
-        if (job.targetPartitions != null) {
-          workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TargetPartitions.name(),
-              Joiner.on(",").join(job.targetPartitions));
-        }
-        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.MaxAttemptsPerTask.name(),
-            String.valueOf(job.maxAttemptsPerTask));
-        workflowBuilder.addConfig(job.name,
-            JobConfig.JobConfigProperty.MaxForcedReassignmentsPerTask.name(),
-            String.valueOf(job.maxForcedReassignmentsPerTask));
-        workflowBuilder.addConfig(job.name,
-            JobConfig.JobConfigProperty.ConcurrentTasksPerInstance.name(),
-            String.valueOf(job.numConcurrentTasksPerInstance));
-        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.TimeoutPerPartition.name(),
-            String.valueOf(job.timeoutPerPartition));
-        workflowBuilder.addConfig(job.name, JobConfig.JobConfigProperty.FailureThreshold.name(),
-            String.valueOf(job.failureThreshold));
-        if (job.tasks != null) {
-          List<TaskConfig> taskConfigs = Lists.newArrayList();
-          for (TaskBean task : job.tasks) {
-            taskConfigs.add(TaskConfig.Builder.from(task));
-          }
-          workflowBuilder.addTaskConfigs(job.name, taskConfigs);
-        }
       }
     }
 
-    WorkflowConfig.Builder workflowCfgBuilder = new WorkflowConfig.Builder();
-    if (wf.schedule != null) {
-      workflowCfgBuilder.setScheduleConfig(ScheduleConfig.from(wf.schedule));
-    }
-    workflowCfgBuilder.setExpiry(wf.expiry);
-    workflowBuilder.setWorkflowConfig(workflowCfgBuilder.build());
+    workflowBuilder.setWorkflowConfig(WorkflowConfig.Builder.from(wf).build());
 
     return workflowBuilder.build();
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index db9fdba..844bdf0 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.HelixException;
+import org.apache.helix.task.beans.WorkflowBean;
 import org.apache.log4j.Logger;
 
 /**
@@ -377,6 +378,16 @@ public class WorkflowConfig {
       return _taskDag;
     }
 
+    public static Builder from(WorkflowBean workflowBean) {
+      WorkflowConfig.Builder b = new WorkflowConfig.Builder();
+      if (workflowBean.schedule != null) {
+        b.setScheduleConfig(ScheduleConfig.from(workflowBean.schedule));
+      }
+      b.setExpiry(workflowBean.expiry);
+
+      return b;
+    }
+
     private void validate() {
       if (_expiry < 0) {
         throw new IllegalArgumentException(String

http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
index 8f97cce..2d4ca75 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowRebalancer.java
@@ -372,7 +372,7 @@ public class WorkflowRebalancer extends TaskRebalancer {
           taskConfigs.add(taskConfig);
         }
         jobCfgBuilder.addTaskConfigs(taskConfigs);
-        workflowBuilder.addJobConfig(job, jobCfgBuilder);
+        workflowBuilder.addJob(job, jobCfgBuilder);
 
         // Add dag dependencies
         Set<String> children = parentsToChildren.get(namespacedJob);

http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java b/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
index 32fd5ac..a570026 100644
--- a/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
+++ b/helix-core/src/main/java/org/apache/helix/task/beans/JobBean.java
@@ -34,7 +34,7 @@ public class JobBean {
   public List<String> targetPartitionStates;
   public List<String> targetPartitions;
   public String command;
-  public Map<String, String> jobConfigMap;
+  public Map<String, String> jobCommandConfigMap;
   public List<TaskBean> tasks;
   public long timeoutPerPartition = JobConfig.DEFAULT_TIMEOUT_PER_TASK;
   public int numConcurrentTasksPerInstance = JobConfig.DEFAULT_NUM_CONCURRENT_TASKS_PER_INSTANCE;
@@ -42,4 +42,6 @@ public class JobBean {
   public int maxForcedReassignmentsPerTask = JobConfig.DEFAULT_MAX_FORCED_REASSIGNMENTS_PER_TASK;
   public int failureThreshold = JobConfig.DEFAULT_FAILURE_THRESHOLD;
   public long taskRetryDelay = JobConfig.DEFAULT_TASK_RETRY_DELAY;
+  public boolean disableExternalView = JobConfig.DEFAULT_DISABLE_EXTERNALVIEW;
+  public boolean ignoreDependentJobFailure = JobConfig.DEFAULT_IGNORE_DEPENDENT_JOB_FAILURE;
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
index 1c58776..046281e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
@@ -152,7 +152,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     JobConfig.Builder jobBuilder =
         new JobConfig.Builder().setCommand("DummyCommand").addTaskConfigs(taskConfigs)
             .setJobCommandConfigMap(jobCommandMap);
-    workflowBuilder.addJobConfig(jobName, jobBuilder);
+    workflowBuilder.addJob(jobName, jobBuilder);
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
@@ -179,7 +179,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     JobConfig.Builder jobBuilder =
         new JobConfig.Builder().setCommand("DummyCommand").setFailureThreshold(1)
             .addTaskConfigs(taskConfigs).setJobCommandConfigMap(jobConfigMap);
-    workflowBuilder.addJobConfig(jobName, jobBuilder);
+    workflowBuilder.addJob(jobName, jobBuilder);
     _driver.start(workflowBuilder.build());
 
     // Ensure the job completes
@@ -207,7 +207,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     JobConfig.Builder jobBuilder =
         new JobConfig.Builder().setCommand("DummyCommand").addTaskConfigs(taskConfigs)
             .setJobCommandConfigMap(jobCommandMap);
-    workflowBuilder.addJobConfig(jobName, jobBuilder);
+    workflowBuilder.addJob(jobName, jobBuilder);
 
     _driver.start(workflowBuilder.build());
 
@@ -235,7 +235,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand("DummyCommand")
         .setMaxForcedReassignmentsPerTask(NUM_INSTANCES - 1).addTaskConfigs(taskConfigs)
         .setJobCommandConfigMap(jobCommandMap);
-    workflowBuilder.addJobConfig(jobName, jobBuilder);
+    workflowBuilder.addJob(jobName, jobBuilder);
 
     _driver.start(workflowBuilder.build());
 
@@ -268,7 +268,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand("DummyCommand")
         .addTaskConfigs(taskConfigs)
         .setJobCommandConfigMap(jobCommandMap);
-    workflowBuilder.addJobConfig(jobName, jobBuilder);
+    workflowBuilder.addJob(jobName, jobBuilder);
 
     long inFiveSeconds = System.currentTimeMillis() + (5 * 1000);
     workflowBuilder.setScheduleConfig(ScheduleConfig.oneTimeDelayedStart(new Date(inFiveSeconds)));
@@ -301,7 +301,7 @@ public class TestIndependentTaskRebalancer extends ZkIntegrationTestBase {
     JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand("DummyCommand")
         .setTaskRetryDelay(delay).addTaskConfigs(taskConfigs)
         .setJobCommandConfigMap(jobCommandMap);
-    workflowBuilder.addJobConfig(jobName, jobBuilder);
+    workflowBuilder.addJob(jobName, jobBuilder);
 
     SingleFailTask.hasFailed = false;
     _driver.start(workflowBuilder.build());

http://git-wip-us.apache.org/repos/asf/helix/blob/1f683b86/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java b/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
index 639cdff..d428460 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
@@ -70,8 +70,8 @@ public class WorkflowGenerator {
     JobConfig.Builder jobBuilder = JobConfig.Builder.fromMap(DEFAULT_JOB_CONFIG);
     jobBuilder.setJobCommandConfigMap(DEFAULT_COMMAND_CONFIG);
 
-    builder.addJobConfig(JOB_NAME_1, jobBuilder);
-    builder.addJobConfig(JOB_NAME_2, jobBuilder);
+    builder.addJob(JOB_NAME_1, jobBuilder);
+    builder.addJob(JOB_NAME_2, jobBuilder);
 
     return builder;
   }


[10/33] helix git commit: Add old task rebalancers back for back-compatible and rolling upgrade.

Posted by lx...@apache.org.
Add old task rebalancers back for back-compatible and rolling upgrade.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/66dba1f5
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/66dba1f5
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/66dba1f5

Branch: refs/heads/helix-0.6.x
Commit: 66dba1f5df1d2d72a40aabfa6c841152cc068a70
Parents: 1dad0b8
Author: Lei Xia <lx...@linkedin.com>
Authored: Mon Mar 7 18:10:34 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:47:21 2016 -0700

----------------------------------------------------------------------
 .../stages/BestPossibleStateCalcStage.java      |    2 +-
 .../java/org/apache/helix/model/IdealState.java |    6 +-
 .../helix/task/DeprecatedTaskRebalancer.java    | 1134 ++++++++++++++++++
 .../helix/task/FixedTargetTaskRebalancer.java   |   58 +
 .../helix/task/GenericTaskRebalancer.java       |   57 +
 .../java/org/apache/helix/task/TaskDriver.java  |   31 +-
 .../java/org/apache/helix/task/Workflow.java    |    4 +-
 7 files changed, 1281 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index f12b6e5..b24507c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -133,7 +133,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
               Rebalancer.class.cast(HelixUtil.loadClass(getClass(), rebalancerClassName)
                   .newInstance());
         } catch (Exception e) {
-          logger.warn("Exception while invoking custom rebalancer class:" + rebalancerClassName, e);
+          logger.error("Exception while invoking custom rebalancer class:" + rebalancerClassName, e);
         }
         if (rebalancer != null) {
           try {

http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/model/IdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/IdealState.java b/helix-core/src/main/java/org/apache/helix/model/IdealState.java
index e7f6096..44f4219 100644
--- a/helix-core/src/main/java/org/apache/helix/model/IdealState.java
+++ b/helix-core/src/main/java/org/apache/helix/model/IdealState.java
@@ -31,6 +31,8 @@ import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.rebalancer.Rebalancer;
+import org.apache.helix.task.FixedTargetTaskRebalancer;
+import org.apache.helix.task.GenericTaskRebalancer;
 import org.apache.helix.task.JobRebalancer;
 import org.apache.helix.task.TaskRebalancer;
 import org.apache.helix.task.WorkflowRebalancer;
@@ -525,7 +527,9 @@ public class IdealState extends HelixProperty {
       String rebalancerName = getRebalancerClassName();
       if (rebalancerName != null) {
         if (rebalancerName.equals(JobRebalancer.class.getName())
-            || rebalancerName.equals(WorkflowRebalancer.class.getName())) {
+            || rebalancerName.equals(WorkflowRebalancer.class.getName())
+            || rebalancerName.equals(GenericTaskRebalancer.class.getName())
+            || rebalancerName.equals(FixedTargetTaskRebalancer.class.getName())) {
           property = RebalanceMode.TASK;
         } else {
           property = RebalanceMode.USER_DEFINED;

http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
new file mode 100644
index 0000000..6f744f0
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/DeprecatedTaskRebalancer.java
@@ -0,0 +1,1134 @@
+package org.apache.helix.task;
+
+/*
+ * 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.
+ */
+
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.Lists;
+import org.I0Itec.zkclient.DataUpdater;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.controller.rebalancer.Rebalancer;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.log4j.Logger;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+/**
+ * Custom rebalancer implementation for the {@code Task} state model.
+ */
+/** This rebalancer is deprecated, left here only for back-compatible. **/
+@Deprecated
+public abstract class DeprecatedTaskRebalancer implements Rebalancer, MappingCalculator {
+  private static final Logger LOG = Logger.getLogger(TaskRebalancer.class);
+
+  // Management of already-scheduled rebalances across jobs
+  private static final BiMap<String, Date> SCHEDULED_TIMES = HashBiMap.create();
+  private static final ScheduledExecutorService SCHEDULED_EXECUTOR = Executors
+      .newSingleThreadScheduledExecutor();
+  public static final String PREV_RA_NODE = "PreviousResourceAssignment";
+
+  // For connection management
+  private HelixManager _manager;
+
+  /**
+   * Get all the partitions that should be created by this task
+   * @param jobCfg the task configuration
+   * @param jobCtx the task context
+   * @param workflowCfg the workflow configuration
+   * @param workflowCtx the workflow context
+   * @param cache cluster snapshot
+   * @return set of partition numbers
+   */
+  public abstract Set<Integer> getAllTaskPartitions(JobConfig jobCfg, JobContext jobCtx,
+      WorkflowConfig workflowCfg, WorkflowContext workflowCtx, ClusterDataCache cache);
+
+  /**
+   * Compute an assignment of tasks to instances
+   * @param currStateOutput the current state of the instances
+   * @param prevAssignment the previous task partition assignment
+   * @param instances the instances
+   * @param jobCfg the task configuration
+   * @param jobContext the task context
+   * @param workflowCfg the workflow configuration
+   * @param workflowCtx the workflow context
+   * @param partitionSet the partitions to assign
+   * @param cache cluster snapshot
+   * @return map of instances to set of partition numbers
+   */
+  public abstract Map<String, SortedSet<Integer>> getTaskAssignment(
+      CurrentStateOutput currStateOutput, ResourceAssignment prevAssignment,
+      Collection<String> instances, JobConfig jobCfg, JobContext jobContext,
+      WorkflowConfig workflowCfg, WorkflowContext workflowCtx, Set<Integer> partitionSet,
+      ClusterDataCache cache);
+
+  @Override
+  public void init(HelixManager manager) {
+    _manager = manager;
+  }
+
+  @Override
+  public ResourceAssignment computeBestPossiblePartitionState(ClusterDataCache clusterData,
+      IdealState taskIs, Resource resource, CurrentStateOutput currStateOutput) {
+    final String resourceName = resource.getResourceName();
+    LOG.debug("Computer Best Partition for resource: " + resourceName);
+
+    // Fetch job configuration
+    JobConfig jobCfg = TaskUtil.getJobCfg(_manager, resourceName);
+    if (jobCfg == null) {
+      LOG.debug("Job configuration is NULL for " + resourceName);
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+    String workflowResource = jobCfg.getWorkflow();
+
+    // Fetch workflow configuration and context
+    WorkflowConfig workflowCfg = TaskUtil.getWorkflowCfg(_manager, workflowResource);
+    if (workflowCfg == null) {
+      LOG.debug("Workflow configuration is NULL for " + resourceName);
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+    WorkflowContext workflowCtx = TaskUtil.getWorkflowContext(_manager, workflowResource);
+
+    // Initialize workflow context if needed
+    if (workflowCtx == null) {
+      workflowCtx = new WorkflowContext(new ZNRecord("WorkflowContext"));
+      workflowCtx.setStartTime(System.currentTimeMillis());
+      LOG.info("Workflow context for " + resourceName + " created!");
+    }
+
+    // check ancestor job status
+    int notStartedCount = 0;
+    int inCompleteCount = 0;
+    for (String ancestor : workflowCfg.getJobDag().getAncestors(resourceName)) {
+      TaskState jobState = workflowCtx.getJobState(ancestor);
+      if (jobState == null || jobState == TaskState.NOT_STARTED) {
+        ++notStartedCount;
+      } else if (jobState == TaskState.IN_PROGRESS || jobState == TaskState.STOPPED) {
+        ++inCompleteCount;
+      }
+    }
+
+    if (notStartedCount > 0 || inCompleteCount >= workflowCfg.getParallelJobs()) {
+      LOG.debug("Job is not ready to be scheduled due to pending dependent jobs " + resourceName);
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+
+    // Clean up if workflow marked for deletion
+    TargetState targetState = workflowCfg.getTargetState();
+    if (targetState == TargetState.DELETE) {
+      LOG.info(
+          "Workflow is marked as deleted " + workflowResource
+              + " cleaning up the workflow context.");
+      cleanup(_manager, resourceName, workflowCfg, workflowResource);
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+
+    // Check if this workflow has been finished past its expiry.
+    if (workflowCtx.getFinishTime() != WorkflowContext.UNFINISHED
+        && workflowCtx.getFinishTime() + workflowCfg.getExpiry() <= System.currentTimeMillis()) {
+      LOG.info("Workflow " + workflowResource
+          + " is completed and passed expiry time, cleaning up the workflow context.");
+      markForDeletion(_manager, workflowResource);
+      cleanup(_manager, resourceName, workflowCfg, workflowResource);
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+
+    // Fetch any existing context information from the property store.
+    JobContext jobCtx = TaskUtil.getJobContext(_manager, resourceName);
+    if (jobCtx == null) {
+      jobCtx = new JobContext(new ZNRecord("TaskContext"));
+      jobCtx.setStartTime(System.currentTimeMillis());
+    }
+
+    // Check for expired jobs for non-terminable workflows
+    long jobFinishTime = jobCtx.getFinishTime();
+    if (!workflowCfg.isTerminable() && jobFinishTime != WorkflowContext.UNFINISHED
+        && jobFinishTime + workflowCfg.getExpiry() <= System.currentTimeMillis()) {
+      LOG.info("Job " + resourceName
+          + " is completed and passed expiry time, cleaning up the job context.");
+      cleanup(_manager, resourceName, workflowCfg, workflowResource);
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+
+    // The job is already in a final state (completed/failed).
+    if (workflowCtx.getJobState(resourceName) == TaskState.FAILED
+        || workflowCtx.getJobState(resourceName) == TaskState.COMPLETED) {
+      LOG.debug("Job " + resourceName + " is failed or already completed.");
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+
+    // Check for readiness, and stop processing if it's not ready
+    boolean isReady =
+        scheduleIfNotReady(workflowCfg, workflowCtx, workflowResource, resourceName, clusterData);
+    if (!isReady) {
+      LOG.debug("Job " + resourceName + " is not ready to be scheduled.");
+      return emptyAssignment(resourceName, currStateOutput);
+    }
+
+    // Grab the old assignment, or an empty one if it doesn't exist
+    ResourceAssignment prevAssignment = getPrevResourceAssignment(_manager, resourceName);
+    if (prevAssignment == null) {
+      prevAssignment = new ResourceAssignment(resourceName);
+    }
+
+    // Will contain the list of partitions that must be explicitly dropped from the ideal state that
+    // is stored in zk.
+    // Fetch the previous resource assignment from the property store. This is required because of
+    // HELIX-230.
+    Set<Integer> partitionsToDrop = new TreeSet<Integer>();
+
+    ResourceAssignment newAssignment =
+        computeResourceMapping(resourceName, workflowCfg, jobCfg, prevAssignment, clusterData
+            .getLiveInstances().keySet(), currStateOutput, workflowCtx, jobCtx, partitionsToDrop,
+            clusterData);
+
+    if (!partitionsToDrop.isEmpty()) {
+      for (Integer pId : partitionsToDrop) {
+        taskIs.getRecord().getMapFields().remove(pName(resourceName, pId));
+      }
+      HelixDataAccessor accessor = _manager.getHelixDataAccessor();
+      PropertyKey propertyKey = accessor.keyBuilder().idealStates(resourceName);
+      accessor.setProperty(propertyKey, taskIs);
+    }
+
+    // Update rebalancer context, previous ideal state.
+    TaskUtil.setJobContext(_manager, resourceName, jobCtx);
+    TaskUtil.setWorkflowContext(_manager, workflowResource, workflowCtx);
+    setPrevResourceAssignment(_manager, resourceName, newAssignment);
+
+    LOG.debug("Job " + resourceName + " new assignment " + Arrays
+        .toString(newAssignment.getMappedPartitions().toArray()));
+
+    return newAssignment;
+  }
+
+  /**
+   * Get the last task assignment for a given job
+   * @param manager a connection to Helix
+   * @param resourceName the name of the job
+   * @return {@link ResourceAssignment} instance, or null if no assignment is available
+   */
+  private ResourceAssignment getPrevResourceAssignment(HelixManager manager,
+      String resourceName) {
+    ZNRecord r =
+        manager.getHelixPropertyStore().get(
+            Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, resourceName, PREV_RA_NODE),
+            null, AccessOption.PERSISTENT);
+    return r != null ? new ResourceAssignment(r) : null;
+  }
+
+  /**
+   * Set the last task assignment for a given job
+   * @param manager a connection to Helix
+   * @param resourceName the name of the job
+   * @param ra {@link ResourceAssignment} containing the task assignment
+   */
+  public void setPrevResourceAssignment(HelixManager manager, String resourceName,
+      ResourceAssignment ra) {
+    manager.getHelixPropertyStore().set(
+        Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, resourceName, PREV_RA_NODE),
+        ra.getRecord(), AccessOption.PERSISTENT);
+  }
+
+  private Set<String> getInstancesAssignedToOtherJobs(String currentJobName,
+      WorkflowConfig workflowCfg) {
+
+    Set<String> ret = new HashSet<String>();
+
+    for (String jobName : workflowCfg.getJobDag().getAllNodes()) {
+      if (jobName.equals(currentJobName)) {
+        continue;
+      }
+
+      JobContext jobContext = TaskUtil.getJobContext(_manager, jobName);
+      if (jobContext == null) {
+        continue;
+      }
+      for (int partition : jobContext.getPartitionSet()) {
+        TaskPartitionState partitionState = jobContext.getPartitionState(partition);
+        if (partitionState == TaskPartitionState.INIT ||
+            partitionState == TaskPartitionState.RUNNING) {
+          ret.add(jobContext.getAssignedParticipant(partition));
+        }
+      }
+    }
+
+    return ret;
+  }
+
+  private ResourceAssignment computeResourceMapping(String jobResource,
+      WorkflowConfig workflowConfig, JobConfig jobCfg, ResourceAssignment prevAssignment,
+      Collection<String> liveInstances, CurrentStateOutput currStateOutput,
+      WorkflowContext workflowCtx, JobContext jobCtx, Set<Integer> partitionsToDropFromIs,
+      ClusterDataCache cache) {
+    TargetState jobTgtState = workflowConfig.getTargetState();
+
+    // Update running status in workflow context
+    if (jobTgtState == TargetState.STOP) {
+      workflowCtx.setJobState(jobResource, TaskState.STOPPED);
+      // Workflow has been stopped if all jobs are stopped
+      if (isWorkflowStopped(workflowCtx, workflowConfig)) {
+        workflowCtx.setWorkflowState(TaskState.STOPPED);
+      }
+    } else {
+      workflowCtx.setJobState(jobResource, TaskState.IN_PROGRESS);
+      // Workflow is in progress if any task is in progress
+      workflowCtx.setWorkflowState(TaskState.IN_PROGRESS);
+    }
+
+    // Used to keep track of tasks that have already been assigned to instances.
+    Set<Integer> assignedPartitions = new HashSet<Integer>();
+
+    // Used to keep track of tasks that have failed, but whose failure is acceptable
+    Set<Integer> skippedPartitions = new HashSet<Integer>();
+
+    // Keeps a mapping of (partition) -> (instance, state)
+    Map<Integer, PartitionAssignment> paMap = new TreeMap<Integer, PartitionAssignment>();
+
+    Set<String> excludedInstances = getInstancesAssignedToOtherJobs(jobResource, workflowConfig);
+
+    // Process all the current assignments of tasks.
+    Set<Integer> allPartitions =
+        getAllTaskPartitions(jobCfg, jobCtx, workflowConfig, workflowCtx, cache);
+    Map<String, SortedSet<Integer>> taskAssignments =
+        getTaskPartitionAssignments(liveInstances, prevAssignment, allPartitions);
+    long currentTime = System.currentTimeMillis();
+    for (String instance : taskAssignments.keySet()) {
+      if (excludedInstances.contains(instance)) {
+        continue;
+      }
+
+      Set<Integer> pSet = taskAssignments.get(instance);
+      // Used to keep track of partitions that are in one of the final states: COMPLETED, TIMED_OUT,
+      // TASK_ERROR, ERROR.
+      Set<Integer> donePartitions = new TreeSet<Integer>();
+      for (int pId : pSet) {
+        final String pName = pName(jobResource, pId);
+
+        // Check for pending state transitions on this (partition, instance).
+        Message pendingMessage =
+            currStateOutput.getPendingState(jobResource, new Partition(pName), instance);
+        if (pendingMessage != null) {
+          // There is a pending state transition for this (partition, instance). Just copy forward
+          // the state assignment from the previous ideal state.
+          Map<String, String> stateMap = prevAssignment.getReplicaMap(new Partition(pName));
+          if (stateMap != null) {
+            String prevState = stateMap.get(instance);
+            paMap.put(pId, new PartitionAssignment(instance, prevState));
+            assignedPartitions.add(pId);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(String
+                  .format(
+                      "Task partition %s has a pending state transition on instance %s. Using the previous ideal state which was %s.",
+                      pName, instance, prevState));
+            }
+          }
+
+          continue;
+        }
+
+        TaskPartitionState currState =
+            TaskPartitionState.valueOf(currStateOutput.getCurrentState(jobResource, new Partition(
+                pName), instance));
+        jobCtx.setPartitionState(pId, currState);
+
+        // Process any requested state transitions.
+        String requestedStateStr =
+            currStateOutput.getRequestedState(jobResource, new Partition(pName), instance);
+        if (requestedStateStr != null && !requestedStateStr.isEmpty()) {
+          TaskPartitionState requestedState = TaskPartitionState.valueOf(requestedStateStr);
+          if (requestedState.equals(currState)) {
+            LOG.warn(String.format(
+                "Requested state %s is the same as the current state for instance %s.",
+                requestedState, instance));
+          }
+
+          paMap.put(pId, new PartitionAssignment(instance, requestedState.name()));
+          assignedPartitions.add(pId);
+          LOG.debug(String.format(
+              "Instance %s requested a state transition to %s for partition %s.", instance,
+              requestedState, pName));
+          continue;
+        }
+
+        switch (currState) {
+        case RUNNING:
+        case STOPPED: {
+          TaskPartitionState nextState;
+          if (jobTgtState == TargetState.START) {
+            nextState = TaskPartitionState.RUNNING;
+          } else {
+            nextState = TaskPartitionState.STOPPED;
+          }
+
+          paMap.put(pId, new PartitionAssignment(instance, nextState.name()));
+          assignedPartitions.add(pId);
+          LOG.debug(String.format("Setting task partition %s state to %s on instance %s.", pName,
+              nextState, instance));
+        }
+          break;
+        case COMPLETED: {
+          // The task has completed on this partition. Mark as such in the context object.
+          donePartitions.add(pId);
+          LOG.debug(String
+              .format(
+                  "Task partition %s has completed with state %s. Marking as such in rebalancer context.",
+                  pName, currState));
+          partitionsToDropFromIs.add(pId);
+          markPartitionCompleted(jobCtx, pId);
+        }
+          break;
+        case TIMED_OUT:
+        case TASK_ERROR:
+        case ERROR: {
+          donePartitions.add(pId); // The task may be rescheduled on a different instance.
+          LOG.debug(String.format(
+              "Task partition %s has error state %s. Marking as such in rebalancer context.",
+              pName, currState));
+          markPartitionError(jobCtx, pId, currState, true);
+          // The error policy is to fail the task as soon a single partition fails for a specified
+          // maximum number of attempts.
+          if (jobCtx.getPartitionNumAttempts(pId) >= jobCfg.getMaxAttemptsPerTask()) {
+            // If the user does not require this task to succeed in order for the job to succeed,
+            // then we don't have to fail the job right now
+            boolean successOptional = false;
+            String taskId = jobCtx.getTaskIdForPartition(pId);
+            if (taskId != null) {
+              TaskConfig taskConfig = jobCfg.getTaskConfig(taskId);
+              if (taskConfig != null) {
+                successOptional = taskConfig.isSuccessOptional();
+              }
+            }
+
+            // Similarly, if we have some leeway for how many tasks we can fail, then we don't have
+            // to fail the job immediately
+            if (skippedPartitions.size() < jobCfg.getFailureThreshold()) {
+              successOptional = true;
+            }
+
+            if (!successOptional) {
+              long finishTime = currentTime;
+              workflowCtx.setJobState(jobResource, TaskState.FAILED);
+              if (workflowConfig.isTerminable()) {
+                workflowCtx.setWorkflowState(TaskState.FAILED);
+                workflowCtx.setFinishTime(finishTime);
+              }
+              jobCtx.setFinishTime(finishTime);
+              markAllPartitionsError(jobCtx, currState, false);
+              addAllPartitions(allPartitions, partitionsToDropFromIs);
+              return emptyAssignment(jobResource, currStateOutput);
+            } else {
+              skippedPartitions.add(pId);
+              partitionsToDropFromIs.add(pId);
+            }
+          } else {
+            // Mark the task to be started at some later time (if enabled)
+            markPartitionDelayed(jobCfg, jobCtx, pId);
+          }
+        }
+          break;
+        case INIT:
+        case DROPPED: {
+          // currState in [INIT, DROPPED]. Do nothing, the partition is eligible to be reassigned.
+          donePartitions.add(pId);
+          LOG.debug(String.format(
+              "Task partition %s has state %s. It will be dropped from the current ideal state.",
+              pName, currState));
+        }
+          break;
+        default:
+          throw new AssertionError("Unknown enum symbol: " + currState);
+        }
+      }
+
+      // Remove the set of task partitions that are completed or in one of the error states.
+      pSet.removeAll(donePartitions);
+    }
+
+    // For delayed tasks, trigger a rebalance event for the closest upcoming ready time
+    scheduleForNextTask(jobResource, jobCtx, currentTime);
+
+    if (isJobComplete(jobCtx, allPartitions, skippedPartitions, jobCfg)) {
+      workflowCtx.setJobState(jobResource, TaskState.COMPLETED);
+      jobCtx.setFinishTime(currentTime);
+      if (isWorkflowComplete(workflowCtx, workflowConfig)) {
+        workflowCtx.setWorkflowState(TaskState.COMPLETED);
+        workflowCtx.setFinishTime(currentTime);
+      }
+    }
+
+    // Make additional task assignments if needed.
+    if (jobTgtState == TargetState.START) {
+      // Contains the set of task partitions that must be excluded from consideration when making
+      // any new assignments.
+      // This includes all completed, failed, delayed, and already assigned partitions.
+      Set<Integer> excludeSet = Sets.newTreeSet(assignedPartitions);
+      addCompletedPartitions(excludeSet, jobCtx, allPartitions);
+      addGiveupPartitions(excludeSet, jobCtx, allPartitions, jobCfg);
+      excludeSet.addAll(skippedPartitions);
+      excludeSet.addAll(getNonReadyPartitions(jobCtx, currentTime));
+      // Get instance->[partition, ...] mappings for the target resource.
+      Map<String, SortedSet<Integer>> tgtPartitionAssignments =
+          getTaskAssignment(currStateOutput, prevAssignment, liveInstances, jobCfg, jobCtx,
+              workflowConfig, workflowCtx, allPartitions, cache);
+      for (Map.Entry<String, SortedSet<Integer>> entry : taskAssignments.entrySet()) {
+        String instance = entry.getKey();
+        if (!tgtPartitionAssignments.containsKey(instance) || excludedInstances.contains(instance)) {
+          continue;
+        }
+        // Contains the set of task partitions currently assigned to the instance.
+        Set<Integer> pSet = entry.getValue();
+        int numToAssign = jobCfg.getNumConcurrentTasksPerInstance() - pSet.size();
+        if (numToAssign > 0) {
+          List<Integer> nextPartitions =
+              getNextPartitions(tgtPartitionAssignments.get(instance), excludeSet, numToAssign);
+          for (Integer pId : nextPartitions) {
+            String pName = pName(jobResource, pId);
+            paMap.put(pId, new PartitionAssignment(instance, TaskPartitionState.RUNNING.name()));
+            excludeSet.add(pId);
+            jobCtx.setAssignedParticipant(pId, instance);
+            jobCtx.setPartitionState(pId, TaskPartitionState.INIT);
+            LOG.debug(String.format("Setting task partition %s state to %s on instance %s.", pName,
+                TaskPartitionState.RUNNING, instance));
+          }
+        }
+      }
+    }
+
+    // Construct a ResourceAssignment object from the map of partition assignments.
+    ResourceAssignment ra = new ResourceAssignment(jobResource);
+    for (Map.Entry<Integer, PartitionAssignment> e : paMap.entrySet()) {
+      PartitionAssignment pa = e.getValue();
+      ra.addReplicaMap(new Partition(pName(jobResource, e.getKey())),
+          ImmutableMap.of(pa._instance, pa._state));
+    }
+
+    return ra;
+  }
+
+  /**
+   * Check if a workflow is ready to schedule, and schedule a rebalance if it is not
+   * @param workflowCfg the workflow to check
+   * @param workflowCtx the current workflow context
+   * @param workflowResource the Helix resource associated with the workflow
+   * @param jobResource a job from the workflow
+   * @param cache the current snapshot of the cluster
+   * @return true if ready, false if not ready
+   */
+  private boolean scheduleIfNotReady(WorkflowConfig workflowCfg, WorkflowContext workflowCtx,
+      String workflowResource, String jobResource, ClusterDataCache cache) {
+    // Ignore non-scheduled workflows
+    if (workflowCfg == null || workflowCfg.getScheduleConfig() == null) {
+      return true;
+    }
+
+    // Figure out when this should be run, and if it's ready, then just run it
+    ScheduleConfig scheduleConfig = workflowCfg.getScheduleConfig();
+    Date startTime = scheduleConfig.getStartTime();
+    long currentTime = new Date().getTime();
+    long delayFromStart = startTime.getTime() - currentTime;
+
+    if (delayFromStart <= 0) {
+      // Remove any timers that are past-time for this workflow
+      Date scheduledTime = SCHEDULED_TIMES.get(workflowResource);
+      if (scheduledTime != null && currentTime > scheduledTime.getTime()) {
+        LOG.debug("Remove schedule timer for " + jobResource + " time: " + SCHEDULED_TIMES.get(jobResource));
+        SCHEDULED_TIMES.remove(workflowResource);
+      }
+
+      // Recurring workflows are just templates that spawn new workflows
+      if (scheduleConfig.isRecurring()) {
+        // Skip scheduling this workflow if it's not in a start state
+        if (!workflowCfg.getTargetState().equals(TargetState.START)) {
+          LOG.debug(
+              "Skip scheduling since the workflow has not been started " + workflowResource);
+          return false;
+        }
+
+        // Skip scheduling this workflow again if the previous run (if any) is still active
+        String lastScheduled = workflowCtx.getLastScheduledSingleWorkflow();
+        if (lastScheduled != null) {
+          WorkflowContext lastWorkflowCtx = TaskUtil.getWorkflowContext(_manager, lastScheduled);
+          if (lastWorkflowCtx != null
+              && lastWorkflowCtx.getFinishTime() == WorkflowContext.UNFINISHED) {
+            LOG.info("Skip scheduling since last schedule has not completed yet " + lastScheduled);
+            return false;
+          }
+        }
+
+        // Figure out how many jumps are needed, thus the time to schedule the next workflow
+        // The negative of the delay is the amount of time past the start time
+        long period =
+            scheduleConfig.getRecurrenceUnit().toMillis(scheduleConfig.getRecurrenceInterval());
+        long offsetMultiplier = (-delayFromStart) / period;
+        long timeToSchedule = period * offsetMultiplier + startTime.getTime();
+
+        // Now clone the workflow if this clone has not yet been created
+        DateFormat df = new SimpleDateFormat("yyyyMMdd'T'HHmmssZ");
+        // Now clone the workflow if this clone has not yet been created
+        String newWorkflowName = workflowResource + "_" + df.format(new java.util.Date(timeToSchedule));
+        LOG.debug("Ready to start workflow " + newWorkflowName);
+        if (!newWorkflowName.equals(lastScheduled)) {
+          Workflow clonedWf =
+              cloneWorkflow(_manager, workflowResource, newWorkflowName, new Date(timeToSchedule));
+          TaskDriver driver = new TaskDriver(_manager);
+          try {
+            // Start the cloned workflow
+            driver.start(clonedWf);
+          } catch (Exception e) {
+            LOG.error("Failed to schedule cloned workflow " + newWorkflowName, e);
+          }
+          // Persist workflow start regardless of success to avoid retrying and failing
+          workflowCtx.setLastScheduledSingleWorkflow(newWorkflowName);
+          TaskUtil.setWorkflowContext(_manager, workflowResource, workflowCtx);
+        }
+
+        // Change the time to trigger the pipeline to that of the next run
+        startTime = new Date(timeToSchedule + period);
+        delayFromStart = startTime.getTime() - System.currentTimeMillis();
+      } else {
+        // This is a one-time workflow and is ready
+        return true;
+      }
+    }
+
+    scheduleRebalance(workflowResource, jobResource, startTime, delayFromStart);
+    return false;
+  }
+
+  /**
+   * Create a new workflow based on an existing one
+   * @param manager connection to Helix
+   * @param origWorkflowName the name of the existing workflow
+   * @param newWorkflowName the name of the new workflow
+   * @param newStartTime a provided start time that deviates from the desired start time
+   * @return the cloned workflow, or null if there was a problem cloning the existing one
+   */
+  private Workflow cloneWorkflow(HelixManager manager, String origWorkflowName,
+      String newWorkflowName, Date newStartTime) {
+    // Read all resources, including the workflow and jobs of interest
+    HelixDataAccessor accessor = manager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    Map<String, HelixProperty> resourceConfigMap =
+        accessor.getChildValuesMap(keyBuilder.resourceConfigs());
+    if (!resourceConfigMap.containsKey(origWorkflowName)) {
+      LOG.error("No such workflow named " + origWorkflowName);
+      return null;
+    }
+    if (resourceConfigMap.containsKey(newWorkflowName)) {
+      LOG.error("Workflow with name " + newWorkflowName + " already exists!");
+      return null;
+    }
+
+    // Create a new workflow with a new name
+    HelixProperty workflowConfig = resourceConfigMap.get(origWorkflowName);
+    Map<String, String> wfSimpleFields = workflowConfig.getRecord().getSimpleFields();
+    JobDag jobDag = JobDag.fromJson(wfSimpleFields.get(WorkflowConfig.DAG));
+    Map<String, Set<String>> parentsToChildren = jobDag.getParentsToChildren();
+    Workflow.Builder builder = new Workflow.Builder(newWorkflowName);
+
+    // Set the workflow expiry
+    builder.setExpiry(Long.parseLong(wfSimpleFields.get(WorkflowConfig.EXPIRY)));
+
+    // Set the schedule, if applicable
+    ScheduleConfig scheduleConfig;
+    if (newStartTime != null) {
+      scheduleConfig = ScheduleConfig.oneTimeDelayedStart(newStartTime);
+    } else {
+      scheduleConfig = TaskUtil.parseScheduleFromConfigMap(wfSimpleFields);
+    }
+    if (scheduleConfig != null) {
+      builder.setScheduleConfig(scheduleConfig);
+    }
+
+    // Add each job back as long as the original exists
+    Set<String> namespacedJobs = jobDag.getAllNodes();
+    for (String namespacedJob : namespacedJobs) {
+      if (resourceConfigMap.containsKey(namespacedJob)) {
+        // Copy over job-level and task-level configs
+        String job = TaskUtil.getDenamespacedJobName(origWorkflowName, namespacedJob);
+        HelixProperty jobConfig = resourceConfigMap.get(namespacedJob);
+        Map<String, String> jobSimpleFields = jobConfig.getRecord().getSimpleFields();
+        jobSimpleFields.put(JobConfig.JobConfigProperty.WORKFLOW_ID.value(), newWorkflowName); // overwrite workflow name
+        for (Map.Entry<String, String> e : jobSimpleFields.entrySet()) {
+          builder.addConfig(job, e.getKey(), e.getValue());
+        }
+        Map<String, Map<String, String>> rawTaskConfigMap = jobConfig.getRecord().getMapFields();
+        List<TaskConfig> taskConfigs = Lists.newLinkedList();
+        for (Map<String, String> rawTaskConfig : rawTaskConfigMap.values()) {
+          TaskConfig taskConfig = TaskConfig.from(rawTaskConfig);
+          taskConfigs.add(taskConfig);
+        }
+        builder.addTaskConfigs(job, taskConfigs);
+
+        // Add dag dependencies
+        Set<String> children = parentsToChildren.get(namespacedJob);
+        if (children != null) {
+          for (String namespacedChild : children) {
+            String child = TaskUtil.getDenamespacedJobName(origWorkflowName, namespacedChild);
+            builder.addParentChildDependency(job, child);
+          }
+        }
+      }
+    }
+    return builder.build();
+  }
+
+  private void scheduleRebalance(String id, String jobResource, Date startTime, long delayFromStart) {
+    // Do nothing if there is already a timer set for the this workflow with the same start time.
+    if ((SCHEDULED_TIMES.containsKey(id) && SCHEDULED_TIMES.get(id).equals(startTime))
+        || SCHEDULED_TIMES.inverse().containsKey(startTime)) {
+      LOG.debug("Schedule timer for" + id + "and job: " + jobResource + " is up to date.");
+      return;
+    }
+    LOG.info(
+        "Schedule rebalance with id: " + id + "and job: " + jobResource + " at time: " + startTime
+            + " delay from start: " + delayFromStart);
+
+    // For workflows not yet scheduled, schedule them and record it
+    RebalanceInvoker rebalanceInvoker = new RebalanceInvoker(_manager, jobResource);
+    SCHEDULED_TIMES.put(id, startTime);
+    SCHEDULED_EXECUTOR.schedule(rebalanceInvoker, delayFromStart, TimeUnit.MILLISECONDS);
+  }
+
+  private void scheduleForNextTask(String jobResource, JobContext ctx, long now) {
+    // Clear current entries if they exist and are expired
+    long currentTime = now;
+    Date scheduledTime = SCHEDULED_TIMES.get(jobResource);
+    if (scheduledTime != null && currentTime > scheduledTime.getTime()) {
+      LOG.debug("Remove schedule timer for" + jobResource + " time: " + SCHEDULED_TIMES.get(jobResource));
+      SCHEDULED_TIMES.remove(jobResource);
+    }
+
+    // Figure out the earliest schedulable time in the future of a non-complete job
+    boolean shouldSchedule = false;
+    long earliestTime = Long.MAX_VALUE;
+    for (int p : ctx.getPartitionSet()) {
+      long retryTime = ctx.getNextRetryTime(p);
+      TaskPartitionState state = ctx.getPartitionState(p);
+      state = (state != null) ? state : TaskPartitionState.INIT;
+      Set<TaskPartitionState> errorStates =
+          Sets.newHashSet(TaskPartitionState.ERROR, TaskPartitionState.TASK_ERROR,
+              TaskPartitionState.TIMED_OUT);
+      if (errorStates.contains(state) && retryTime > currentTime && retryTime < earliestTime) {
+        earliestTime = retryTime;
+        shouldSchedule = true;
+      }
+    }
+
+    // If any was found, then schedule it
+    if (shouldSchedule) {
+      long delay = earliestTime - currentTime;
+      Date startTime = new Date(earliestTime);
+      scheduleRebalance(jobResource, jobResource, startTime, delay);
+    }
+  }
+
+  /**
+   * Checks if the job has completed.
+   * @param ctx The rebalancer context.
+   * @param allPartitions The set of partitions to check.
+   * @param skippedPartitions partitions that failed, but whose failure is acceptable
+   * @return true if all task partitions have been marked with status
+   *         {@link TaskPartitionState#COMPLETED} in the rebalancer
+   *         context, false otherwise.
+   */
+  private static boolean isJobComplete(JobContext ctx, Set<Integer> allPartitions,
+      Set<Integer> skippedPartitions, JobConfig cfg) {
+    for (Integer pId : allPartitions) {
+      TaskPartitionState state = ctx.getPartitionState(pId);
+      if (!skippedPartitions.contains(pId) && state != TaskPartitionState.COMPLETED
+          && !isTaskGivenup(ctx, cfg, pId)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Checks if the workflow has completed.
+   * @param ctx Workflow context containing job states
+   * @param cfg Workflow config containing set of jobs
+   * @return returns true if all tasks are {@link TaskState#COMPLETED}, false otherwise.
+   */
+  private static boolean isWorkflowComplete(WorkflowContext ctx, WorkflowConfig cfg) {
+    if (!cfg.isTerminable()) {
+      return false;
+    }
+    for (String job : cfg.getJobDag().getAllNodes()) {
+      if (ctx.getJobState(job) != TaskState.COMPLETED) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Checks if the workflow has been stopped.
+   * @param ctx Workflow context containing task states
+   * @param cfg Workflow config containing set of tasks
+   * @return returns true if all tasks are {@link TaskState#STOPPED}, false otherwise.
+   */
+  private static boolean isWorkflowStopped(WorkflowContext ctx, WorkflowConfig cfg) {
+    for (String job : cfg.getJobDag().getAllNodes()) {
+      if (ctx.getJobState(job) != TaskState.STOPPED && ctx.getJobState(job) != null) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static void markForDeletion(HelixManager mgr, String resourceName) {
+    mgr.getConfigAccessor().set(
+        TaskUtil.getResourceConfigScope(mgr.getClusterName(), resourceName),
+        WorkflowConfig.TARGET_STATE, TargetState.DELETE.name());
+  }
+
+  /**
+   * Cleans up all Helix state associated with this job, wiping workflow-level information if this
+   * is the last remaining job in its workflow, and the workflow is terminable.
+   */
+  private static void cleanup(HelixManager mgr, final String resourceName, WorkflowConfig cfg,
+      String workflowResource) {
+    LOG.info("Cleaning up job: " + resourceName + " in workflow: " + workflowResource);
+    HelixDataAccessor accessor = mgr.getHelixDataAccessor();
+
+    // Remove any DAG references in workflow
+    PropertyKey workflowKey = getConfigPropertyKey(accessor, workflowResource);
+    DataUpdater<ZNRecord> dagRemover = new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        JobDag jobDag = JobDag.fromJson(currentData.getSimpleField(WorkflowConfig.DAG));
+        for (String child : jobDag.getDirectChildren(resourceName)) {
+          jobDag.getChildrenToParents().get(child).remove(resourceName);
+        }
+        for (String parent : jobDag.getDirectParents(resourceName)) {
+          jobDag.getParentsToChildren().get(parent).remove(resourceName);
+        }
+        jobDag.getChildrenToParents().remove(resourceName);
+        jobDag.getParentsToChildren().remove(resourceName);
+        jobDag.getAllNodes().remove(resourceName);
+        try {
+          currentData.setSimpleField(WorkflowConfig.DAG, jobDag.toJson());
+        } catch (Exception e) {
+          LOG.equals("Could not update DAG for job: " + resourceName);
+        }
+        return currentData;
+      }
+    };
+    accessor.getBaseDataAccessor().update(workflowKey.getPath(), dagRemover,
+        AccessOption.PERSISTENT);
+
+    // Delete resource configs.
+    PropertyKey cfgKey = getConfigPropertyKey(accessor, resourceName);
+    if (!accessor.removeProperty(cfgKey)) {
+      throw new RuntimeException(String.format(
+          "Error occurred while trying to clean up job %s. Failed to remove node %s from Helix. Aborting further clean up steps.",
+          resourceName,
+          cfgKey));
+    }
+
+    // Delete property store information for this resource.
+    // For recurring workflow, it's OK if the node doesn't exist.
+    String propStoreKey = getRebalancerPropStoreKey(resourceName);
+    mgr.getHelixPropertyStore().remove(propStoreKey, AccessOption.PERSISTENT);
+
+    // Delete the ideal state itself.
+    PropertyKey isKey = getISPropertyKey(accessor, resourceName);
+    if (!accessor.removeProperty(isKey)) {
+      throw new RuntimeException(String.format(
+          "Error occurred while trying to clean up task %s. Failed to remove node %s from Helix.",
+          resourceName, isKey));
+    }
+
+    // Delete dead external view
+    // because job is already completed, there is no more current state change
+    // thus dead external views removal will not be triggered
+    PropertyKey evKey = accessor.keyBuilder().externalView(resourceName);
+    accessor.removeProperty(evKey);
+
+    LOG.info(String.format("Successfully cleaned up job resource %s.", resourceName));
+
+    boolean lastInWorkflow = true;
+    for (String job : cfg.getJobDag().getAllNodes()) {
+      // check if property store information or resource configs exist for this job
+      if (mgr.getHelixPropertyStore().exists(getRebalancerPropStoreKey(job),
+          AccessOption.PERSISTENT)
+          || accessor.getProperty(getConfigPropertyKey(accessor, job)) != null
+          || accessor.getProperty(getISPropertyKey(accessor, job)) != null) {
+        lastInWorkflow = false;
+        break;
+      }
+    }
+
+    // clean up workflow-level info if this was the last in workflow
+    if (lastInWorkflow && (cfg.isTerminable() || cfg.getTargetState() == TargetState.DELETE)) {
+      // delete workflow config
+      PropertyKey workflowCfgKey = getConfigPropertyKey(accessor, workflowResource);
+      if (!accessor.removeProperty(workflowCfgKey)) {
+        throw new RuntimeException(
+            String
+                .format(
+                    "Error occurred while trying to clean up workflow %s. Failed to remove node %s from Helix. Aborting further clean up steps.",
+                    workflowResource, workflowCfgKey));
+      }
+      // Delete property store information for this workflow
+      String workflowPropStoreKey = getRebalancerPropStoreKey(workflowResource);
+      if (!mgr.getHelixPropertyStore().remove(workflowPropStoreKey, AccessOption.PERSISTENT)) {
+        throw new RuntimeException(
+            String
+                .format(
+                    "Error occurred while trying to clean up workflow %s. Failed to remove node %s from Helix. Aborting further clean up steps.",
+                    workflowResource, workflowPropStoreKey));
+      }
+      // Remove pending timer for this workflow if exists
+      if (SCHEDULED_TIMES.containsKey(workflowResource)) {
+        SCHEDULED_TIMES.remove(workflowResource);
+      }
+    }
+
+  }
+
+  private static String getRebalancerPropStoreKey(String resource) {
+    return Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, resource);
+  }
+
+  private static PropertyKey getISPropertyKey(HelixDataAccessor accessor, String resource) {
+    return accessor.keyBuilder().idealStates(resource);
+  }
+
+  private static PropertyKey getConfigPropertyKey(HelixDataAccessor accessor, String resource) {
+    return accessor.keyBuilder().resourceConfig(resource);
+  }
+
+  private static void addAllPartitions(Set<Integer> toAdd, Set<Integer> destination) {
+    for (Integer pId : toAdd) {
+      destination.add(pId);
+    }
+  }
+
+  private static ResourceAssignment emptyAssignment(String name, CurrentStateOutput currStateOutput) {
+    ResourceAssignment assignment = new ResourceAssignment(name);
+    Set<Partition> partitions = currStateOutput.getCurrentStateMappedPartitions(name);
+    for (Partition partition : partitions) {
+      Map<String, String> currentStateMap = currStateOutput.getCurrentStateMap(name, partition);
+      Map<String, String> replicaMap = Maps.newHashMap();
+      for (String instanceName : currentStateMap.keySet()) {
+        replicaMap.put(instanceName, HelixDefinedState.DROPPED.toString());
+      }
+      assignment.addReplicaMap(partition, replicaMap);
+    }
+    return assignment;
+  }
+
+  private static void addCompletedPartitions(Set<Integer> set, JobContext ctx,
+      Iterable<Integer> pIds) {
+    for (Integer pId : pIds) {
+      TaskPartitionState state = ctx.getPartitionState(pId);
+      if (state == TaskPartitionState.COMPLETED) {
+        set.add(pId);
+      }
+    }
+  }
+
+  private static boolean isTaskGivenup(JobContext ctx, JobConfig cfg, int pId) {
+    return ctx.getPartitionNumAttempts(pId) >= cfg.getMaxAttemptsPerTask();
+  }
+
+  // add all partitions that have been tried maxNumberAttempts
+  private static void addGiveupPartitions(Set<Integer> set, JobContext ctx, Iterable<Integer> pIds,
+      JobConfig cfg) {
+    for (Integer pId : pIds) {
+      if (isTaskGivenup(ctx, cfg, pId)) {
+        set.add(pId);
+      }
+    }
+  }
+
+  private static List<Integer> getNextPartitions(SortedSet<Integer> candidatePartitions,
+      Set<Integer> excluded, int n) {
+    List<Integer> result = new ArrayList<Integer>();
+    for (Integer pId : candidatePartitions) {
+      if (result.size() >= n) {
+        break;
+      }
+
+      if (!excluded.contains(pId)) {
+        result.add(pId);
+      }
+    }
+
+    return result;
+  }
+
+  private static void markPartitionDelayed(JobConfig cfg, JobContext ctx, int p) {
+    long delayInterval = cfg.getTaskRetryDelay();
+    if (delayInterval <= 0) {
+      return;
+    }
+    long nextStartTime = ctx.getPartitionFinishTime(p) + delayInterval;
+    ctx.setNextRetryTime(p, nextStartTime);
+  }
+
+  private static void markPartitionCompleted(JobContext ctx, int pId) {
+    ctx.setPartitionState(pId, TaskPartitionState.COMPLETED);
+    ctx.setPartitionFinishTime(pId, System.currentTimeMillis());
+    ctx.incrementNumAttempts(pId);
+  }
+
+  private static void markPartitionError(JobContext ctx, int pId, TaskPartitionState state,
+      boolean incrementAttempts) {
+    ctx.setPartitionState(pId, state);
+    ctx.setPartitionFinishTime(pId, System.currentTimeMillis());
+    if (incrementAttempts) {
+      ctx.incrementNumAttempts(pId);
+    }
+  }
+
+  private static void markAllPartitionsError(JobContext ctx, TaskPartitionState state,
+      boolean incrementAttempts) {
+    for (int pId : ctx.getPartitionSet()) {
+      markPartitionError(ctx, pId, state, incrementAttempts);
+    }
+  }
+
+  /**
+   * Return the assignment of task partitions per instance.
+   */
+  private static Map<String, SortedSet<Integer>> getTaskPartitionAssignments(
+      Iterable<String> instanceList, ResourceAssignment assignment, Set<Integer> includeSet) {
+    Map<String, SortedSet<Integer>> result = new HashMap<String, SortedSet<Integer>>();
+    for (String instance : instanceList) {
+      result.put(instance, new TreeSet<Integer>());
+    }
+
+    for (Partition partition : assignment.getMappedPartitions()) {
+      int pId = pId(partition.getPartitionName());
+      if (includeSet.contains(pId)) {
+        Map<String, String> replicaMap = assignment.getReplicaMap(partition);
+        for (String instance : replicaMap.keySet()) {
+          SortedSet<Integer> pList = result.get(instance);
+          if (pList != null) {
+            pList.add(pId);
+          }
+        }
+      }
+    }
+    return result;
+  }
+
+  private static Set<Integer> getNonReadyPartitions(JobContext ctx, long now) {
+    Set<Integer> nonReadyPartitions = Sets.newHashSet();
+    for (int p : ctx.getPartitionSet()) {
+      long toStart = ctx.getNextRetryTime(p);
+      if (now < toStart) {
+        nonReadyPartitions.add(p);
+      }
+    }
+    return nonReadyPartitions;
+  }
+
+  /**
+   * Computes the partition name given the resource name and partition id.
+   */
+  protected static String pName(String resource, int pId) {
+    return resource + "_" + pId;
+  }
+
+  /**
+   * Extracts the partition id from the given partition name.
+   */
+  protected static int pId(String pName) {
+    String[] tokens = pName.split("_");
+    return Integer.valueOf(tokens[tokens.length - 1]);
+  }
+
+  /**
+   * An (instance, state) pair.
+   */
+  private static class PartitionAssignment {
+    private final String _instance;
+    private final String _state;
+
+    private PartitionAssignment(String instance, String state) {
+      _instance = instance;
+      _state = state;
+    }
+  }
+
+  @Override
+  public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState,
+      CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) {
+    // All of the heavy lifting is in the ResourceAssignment computation,
+    // so this part can just be a no-op.
+    return currentIdealState;
+  }
+
+  /**
+   * The simplest possible runnable that will trigger a run of the controller pipeline
+   */
+  private static class RebalanceInvoker implements Runnable {
+    private final HelixManager _manager;
+    private final String _resource;
+
+    public RebalanceInvoker(HelixManager manager, String resource) {
+      _manager = manager;
+      _resource = resource;
+    }
+
+    @Override
+    public void run() {
+      TaskUtil.invokeRebalance(_manager.getHelixDataAccessor(), _resource);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
new file mode 100644
index 0000000..569fe03
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
@@ -0,0 +1,58 @@
+package org.apache.helix.task;
+
+/*
+ * 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.
+ */
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.ResourceAssignment;
+/**
+ * A rebalancer for when a task group must be assigned according to partitions/states on a target
+ * resource. Here, tasks are colocated according to where a resource's partitions are, as well as
+ * (if desired) only where those partitions are in a given state.
+ */
+
+/**
+ * This rebalancer is deprecated, left here only for back-compatible. *
+ */
+@Deprecated public class FixedTargetTaskRebalancer extends DeprecatedTaskRebalancer {
+  private FixedTargetTaskAssignmentCalculator taskAssignmentCalculator =
+      new FixedTargetTaskAssignmentCalculator();
+
+  @Override public Set<Integer> getAllTaskPartitions(JobConfig jobCfg, JobContext jobCtx,
+      WorkflowConfig workflowCfg, WorkflowContext workflowCtx, ClusterDataCache cache) {
+    return taskAssignmentCalculator
+        .getAllTaskPartitions(jobCfg, jobCtx, workflowCfg, workflowCtx, cache);
+  }
+
+  @Override public Map<String, SortedSet<Integer>> getTaskAssignment(
+      CurrentStateOutput currStateOutput, ResourceAssignment prevAssignment,
+      Collection<String> instances, JobConfig jobCfg, JobContext jobContext,
+      WorkflowConfig workflowCfg, WorkflowContext workflowCtx, Set<Integer> partitionSet,
+      ClusterDataCache cache) {
+    return taskAssignmentCalculator
+        .getTaskAssignment(currStateOutput, prevAssignment, instances, jobCfg, jobContext,
+            workflowCfg, workflowCtx, partitionSet, cache);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/task/GenericTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/GenericTaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/GenericTaskRebalancer.java
new file mode 100644
index 0000000..6a005b9
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/GenericTaskRebalancer.java
@@ -0,0 +1,57 @@
+package org.apache.helix.task;
+
+/*
+ * 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.
+ */
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.ResourceAssignment;
+
+
+/**
+ * This class does an assignment based on an automatic rebalancing strategy, rather than requiring
+ * assignment to target partitions and states of another resource
+ */
+/** This rebalancer is deprecated, left here only for back-compatible. **/
+@Deprecated
+public class GenericTaskRebalancer extends DeprecatedTaskRebalancer {
+  private GenericTaskAssignmentCalculator taskAssignmentCalculator =
+      new GenericTaskAssignmentCalculator();
+
+  @Override
+  public Set<Integer> getAllTaskPartitions(JobConfig jobCfg, JobContext jobCtx,
+      WorkflowConfig workflowCfg, WorkflowContext workflowCtx, ClusterDataCache cache) {
+    return taskAssignmentCalculator
+        .getAllTaskPartitions(jobCfg, jobCtx, workflowCfg, workflowCtx, cache);
+  }
+
+  @Override
+  public Map<String, SortedSet<Integer>> getTaskAssignment(CurrentStateOutput currStateOutput,
+      ResourceAssignment prevAssignment, Collection<String> instances, JobConfig jobCfg,
+      final JobContext jobContext, WorkflowConfig workflowCfg, WorkflowContext workflowCtx,
+      Set<Integer> partitionSet, ClusterDataCache cache) {
+    return taskAssignmentCalculator
+        .getTaskAssignment(currStateOutput, prevAssignment, instances, jobCfg, jobContext,
+            workflowCfg, workflowCtx, partitionSet, cache);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index b55d9d0..c3eb8bd 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -574,6 +574,10 @@ public class TaskDriver {
     if (!status) {
       throw new IllegalArgumentException("Could not enqueue job");
     }
+
+    // This is to make it back-compatible with old Helix task driver.
+    addWorkflowResourceIfNecessary(queueName);
+
     // Schedule the job
     TaskUtil.invokeRebalance(_accessor, queueName);
   }
@@ -583,21 +587,34 @@ public class TaskDriver {
     // Add workflow resource
     _admin.addResource(_clusterName, workflow, 1, TaskConstants.STATE_MODEL_NAME);
 
-    // Push out new ideal state for the workflow
-    CustomModeISBuilder IsBuilder = new CustomModeISBuilder(workflow);
-    IsBuilder.setRebalancerMode(IdealState.RebalanceMode.TASK)
-        .setNumReplica(1).setNumPartitions(1)
-        .setStateModel(TaskConstants.STATE_MODEL_NAME)
-        .setDisableExternalView(true);
+    IdealState is = buildWorkflowIdealState(workflow);
+    _admin.setResourceIdealState(_clusterName, workflow, is);
+
+  }
+
+  /**
+   * Posts new workflow resource to cluster if it does not exist
+   */
+  private void addWorkflowResourceIfNecessary(String workflow) {
+    IdealState is = _admin.getResourceIdealState(_clusterName, workflow);
+    if (is == null) {
+      addWorkflowResource(workflow);
+    }
+  }
 
+  private IdealState buildWorkflowIdealState(String workflow) {
+    CustomModeISBuilder IsBuilder = new CustomModeISBuilder(workflow);
+    IsBuilder.setRebalancerMode(IdealState.RebalanceMode.TASK).setNumReplica(1).setNumPartitions(1)
+        .setStateModel(TaskConstants.STATE_MODEL_NAME).setDisableExternalView(true);
     IdealState is = IsBuilder.build();
     is.getRecord().setListField(workflow, new ArrayList<String>());
     is.getRecord().setMapField(workflow, new HashMap<String, String>());
     is.setRebalancerClassName(WorkflowRebalancer.class.getName());
-    _admin.setResourceIdealState(_clusterName, workflow, is);
 
+    return is;
   }
 
+
   /**
    * Add new job config to cluster
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/66dba1f5/helix-core/src/main/java/org/apache/helix/task/Workflow.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/Workflow.java b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
index 3a050c2..1706bec 100644
--- a/helix-core/src/main/java/org/apache/helix/task/Workflow.java
+++ b/helix-core/src/main/java/org/apache/helix/task/Workflow.java
@@ -247,7 +247,7 @@ public class Workflow {
       _expiry = -1;
     }
 
-    private Builder addConfig(String job, String key, String val) {
+    protected Builder addConfig(String job, String key, String val) {
       job = namespacify(job);
       _dag.addNode(job);
       if (!_jobConfigs.containsKey(job)) {
@@ -273,7 +273,7 @@ public class Workflow {
       return this;
     }
 
-    private Builder addTaskConfigs(String job, Collection<TaskConfig> taskConfigs) {
+    protected Builder addTaskConfigs(String job, Collection<TaskConfig> taskConfigs) {
       job = namespacify(job);
       _dag.addNode(job);
       if (!_taskConfigs.containsKey(job)) {


[06/33] helix git commit: TaskUtil.getWorkflowCfg throws NPE if workflow doesn't exist.

Posted by lx...@apache.org.
TaskUtil.getWorkflowCfg throws NPE if workflow doesn't exist.


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

Branch: refs/heads/helix-0.6.x
Commit: 6d42db462bc65ed9f94f22ca6e2de83cb703ea87
Parents: d213c1a
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Feb 23 10:16:03 2016 -0800
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:43:54 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/TaskDriver.java  | 15 +++----
 .../java/org/apache/helix/task/TaskUtil.java    | 41 ++++++--------------
 .../integration/task/TestRecurringJobQueue.java | 16 ++++++++
 3 files changed, 36 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/6d42db46/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index ce7bbf0..b55d9d0 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -222,7 +222,8 @@ public class TaskDriver {
    *
    * Example:
    *
-   * WorkflowConfig currentWorkflowConfig = TaskUtil.getWorkflowCfg(_manager, workflow);
+   * _driver = new TaskDriver ...
+   * WorkflowConfig currentWorkflowConfig = _driver.getWorkflowCfg(_manager, workflow);
    * WorkflowConfig.Builder configBuilder = new WorkflowConfig.Builder(currentWorkflowConfig);
 
    * // make needed changes to the config here
@@ -236,7 +237,7 @@ public class TaskDriver {
    */
   public void updateWorkflow(String workflow, WorkflowConfig newWorkflowConfig) {
     WorkflowConfig currentConfig =
-        TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, workflow);
+        TaskUtil.getWorkflowCfg(_accessor, workflow);
     if (currentConfig == null) {
       throw new HelixException("Workflow " + workflow + " does not exist!");
     }
@@ -270,7 +271,7 @@ public class TaskDriver {
   // TODO: need to make sure the queue is stopped or completed before flush the queue.
   public void flushQueue(String queueName) {
     WorkflowConfig config =
-        TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, queueName);
+        TaskUtil.getWorkflowCfg(_accessor, queueName);
     if (config == null) {
       throw new IllegalArgumentException("Queue does not exist!");
     }
@@ -339,7 +340,7 @@ public class TaskDriver {
    */
   public void deleteJob(final String queueName, final String jobName) {
     WorkflowConfig workflowCfg =
-        TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, queueName);
+        TaskUtil.getWorkflowCfg(_accessor, queueName);
 
     if (workflowCfg == null) {
       throw new IllegalArgumentException("Queue " + queueName + " does not yet exist!");
@@ -384,7 +385,7 @@ public class TaskDriver {
    */
   private void deleteJobFromScheduledQueue(final String queueName, final String jobName) {
     WorkflowConfig workflowCfg =
-        TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, queueName);
+        TaskUtil.getWorkflowCfg(_accessor, queueName);
 
     if (workflowCfg == null) {
       throw new IllegalArgumentException("Queue " + queueName + " does not yet exist!");
@@ -696,7 +697,7 @@ public class TaskDriver {
   }
 
   public WorkflowConfig getWorkflowConfig(String workflow) {
-    return TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, workflow);
+    return TaskUtil.getWorkflowCfg(_accessor, workflow);
   }
 
   public WorkflowContext getWorkflowContext(String workflow) {
@@ -712,7 +713,7 @@ public class TaskDriver {
   }
 
   public void list(String resource) {
-    WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_cfgAccessor, _accessor, _clusterName, resource);
+    WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_accessor, resource);
     if (wCfg == null) {
       LOG.error("Workflow " + resource + " does not exist!");
       return;

http://git-wip-us.apache.org/repos/asf/helix/blob/6d42db46/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
index 524b889..ca274d0 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
@@ -30,7 +30,6 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.helix.AccessOption;
-import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
@@ -40,7 +39,6 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.store.HelixPropertyStore;
 import org.apache.log4j.Logger;
@@ -98,21 +96,19 @@ public class TaskUtil {
   /**
    * Parses workflow resource configurations in Helix into a {@link WorkflowConfig} object.
    *
-   * @param cfgAccessor      Config accessor to access Helix configs
-   * @param accessor         Accessor to access Helix configs
-   * @param clusterName      Cluster name
-   * @param workflowResource The name of the workflow resource.
+   * @param accessor  Accessor to access Helix configs
+   * @param workflow The name of the workflow.
    * @return A {@link WorkflowConfig} object if Helix contains valid configurations for the
    * workflow, null otherwise.
    */
-  public static WorkflowConfig getWorkflowCfg(ConfigAccessor cfgAccessor,
-      HelixDataAccessor accessor, String clusterName, String workflowResource) {
-    Map<String, String> workflowCfg =
-        getResourceConfigMap(cfgAccessor, accessor, clusterName, workflowResource);
+  public static WorkflowConfig getWorkflowCfg(HelixDataAccessor accessor, String workflow) {
+    HelixProperty workflowCfg = getResourceConfig(accessor, workflow);
     if (workflowCfg == null) {
       return null;
     }
-    WorkflowConfig.Builder b = WorkflowConfig.Builder.fromMap(workflowCfg);
+
+    WorkflowConfig.Builder b =
+        WorkflowConfig.Builder.fromMap(workflowCfg.getRecord().getSimpleFields());
 
     return b.build();
   }
@@ -121,13 +117,12 @@ public class TaskUtil {
    * Parses workflow resource configurations in Helix into a {@link WorkflowConfig} object.
    *
    * @param manager          Helix manager object used to connect to Helix.
-   * @param workflowResource The name of the workflow resource.
+   * @param workflow The name of the workflow resource.
    * @return A {@link WorkflowConfig} object if Helix contains valid configurations for the
    * workflow, null otherwise.
    */
-  public static WorkflowConfig getWorkflowCfg(HelixManager manager, String workflowResource) {
-    return getWorkflowCfg(manager.getConfigAccessor(), manager.getHelixDataAccessor(),
-        manager.getClusterName(), workflowResource);
+  public static WorkflowConfig getWorkflowCfg(HelixManager manager, String workflow) {
+    return getWorkflowCfg(manager.getHelixDataAccessor(), workflow);
   }
 
   /**
@@ -452,18 +447,6 @@ public class TaskUtil {
     return workflowBuilder.build();
   }
 
-  private static Map<String, String> getResourceConfigMap(ConfigAccessor cfgAccessor,
-      HelixDataAccessor accessor, String clusterName, String resource) {
-    HelixConfigScope scope = getResourceConfigScope(clusterName, resource);
-
-    List<String> cfgKeys = cfgAccessor.getKeys(scope);
-    if (cfgKeys == null || cfgKeys.isEmpty()) {
-      return null;
-    }
-
-    return getResourceConfig(accessor, resource).getRecord().getSimpleFields();
-  }
-
   private static HelixProperty getResourceConfig(HelixDataAccessor accessor, String resource) {
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
     return accessor.getProperty(keyBuilder.resourceConfig(resource));
@@ -522,7 +505,7 @@ public class TaskUtil {
     return Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, resource);
   }
 
-  public static PropertyKey getWorkflowConfigKey(HelixDataAccessor accessor, String resource) {
-    return accessor.keyBuilder().resourceConfig(resource);
+  public static PropertyKey getWorkflowConfigKey(HelixDataAccessor accessor, String workflow) {
+    return accessor.keyBuilder().resourceConfig(workflow);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/6d42db46/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index cb44f0e..d83c5eb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -47,6 +47,8 @@ import org.apache.helix.task.TaskFactory;
 import org.apache.helix.task.TaskState;
 import org.apache.helix.task.TaskStateModelFactory;
 import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -367,6 +369,20 @@ public class TestRecurringJobQueue extends ZkIntegrationTestBase {
         String.format("%s_%s", scheduledQueue, jobNames.get(JOB_COUNTS - 1)));
   }
 
+  @Test
+  public void testGetNoExistWorkflowConfig() {
+    String randomName = "randomJob";
+    WorkflowConfig workflowConfig = _driver.getWorkflowConfig(randomName);
+    Assert.assertNull(workflowConfig);
+    JobConfig jobConfig = _driver.getJobConfig(randomName);
+    Assert.assertNull(jobConfig);
+    WorkflowContext workflowContext = _driver.getWorkflowContext(randomName);
+    Assert.assertNull(workflowContext);
+    JobContext jobContext = _driver.getJobContext(randomName);
+    Assert.assertNull(jobContext);
+
+  }
+
   private void verifyJobDeleted(String queueName, String jobName) throws Exception {
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();


[32/33] helix git commit: Add integration test for running task with unregistered command. Expected behavior: 1. Task state is "ERROR" 2. No retry will happen.

Posted by lx...@apache.org.
Add integration test for running task with unregistered command. Expected behavior:
1. Task state is "ERROR"
2. No retry will happen.


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/6455b8b9
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/6455b8b9
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/6455b8b9

Branch: refs/heads/helix-0.6.x
Commit: 6455b8b9994178afc5d8a35589456960f0ef264c
Parents: 183a26a
Author: Junkai Xue <jx...@linkedin.com>
Authored: Tue May 24 11:59:08 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:22:15 2016 -0700

----------------------------------------------------------------------
 .../task/TestUnregisteredCommand.java           | 64 ++++++++++++++++++++
 1 file changed, 64 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/6455b8b9/helix-core/src/test/java/org/apache/helix/integration/task/TestUnregisteredCommand.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUnregisteredCommand.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUnregisteredCommand.java
new file mode 100644
index 0000000..4d16a9b
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUnregisteredCommand.java
@@ -0,0 +1,64 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import java.util.Map;
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.TaskPartitionState;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestUnregisteredCommand extends TaskTestBase {
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numNodes = 1;
+    _numReplicas = 1;
+    _numDbs = 1;
+    _numParitions = 1;
+    super.beforeClass();
+  }
+
+  @Test public void testUnregisteredCommand() throws InterruptedException {
+    String workflowName = TestHelper.getTestMethodName();
+    Workflow.Builder builder = new Workflow.Builder(workflowName);
+
+    JobConfig.Builder jobBuilder =
+        new JobConfig.Builder().setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+            .setCommand("OtherCommand").setTimeoutPerTask(10000L).setMaxAttemptsPerTask(2)
+            .setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG);
+
+    builder.addJob("JOB1", jobBuilder);
+
+    _driver.start(builder.build());
+
+    _driver.pollForWorkflowState(workflowName, TaskState.COMPLETED, TaskState.FAILED);
+    Assert.assertEquals(_driver.getJobContext(TaskUtil.getNamespacedJobName(workflowName, "JOB1"))
+        .getPartitionState(0), TaskPartitionState.ERROR);
+    Assert.assertEquals(_driver.getJobContext(TaskUtil.getNamespacedJobName(workflowName, "JOB1"))
+        .getPartitionNumAttempts(0), 1);
+  }
+}


[21/33] helix git commit: Add a new task state (TASK_ABORTED) to TaskResult. This allows client to abort a task and let Helix not retry it even Task.RetryCount is bigger than 1.

Posted by lx...@apache.org.
Add a new task state (TASK_ABORTED) to TaskResult. This allows client to abort a task and let Helix not retry it even Task.RetryCount is bigger than 1.


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

Branch: refs/heads/helix-0.6.x
Commit: 7f184839aebcb817e3c9b0626bcc2a9b44c6926c
Parents: ddc11f9
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Jul 5 15:58:51 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:14:30 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/model/TaskSMD.java    |  38 ++++---
 .../java/org/apache/helix/task/JobConfig.java   |   1 -
 .../org/apache/helix/task/JobRebalancer.java    |  21 ++--
 .../apache/helix/task/TaskPartitionState.java   |   4 +-
 .../java/org/apache/helix/task/TaskResult.java  |  10 +-
 .../java/org/apache/helix/task/TaskRunner.java  |  12 ++-
 .../org/apache/helix/task/TaskStateModel.java   |  32 +++++-
 .../org/apache/helix/task/WorkflowConfig.java   |   2 +-
 .../apache/helix/integration/task/MockTask.java |  35 +++++--
 .../integration/task/TestRecurringJobQueue.java |   5 +-
 .../task/TestTaskConditionalRetry.java          | 102 +++++++++++++++++++
 11 files changed, 226 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/model/TaskSMD.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/TaskSMD.java b/helix-core/src/main/java/org/apache/helix/model/TaskSMD.java
index d826358..fd8b585 100644
--- a/helix-core/src/main/java/org/apache/helix/model/TaskSMD.java
+++ b/helix-core/src/main/java/org/apache/helix/model/TaskSMD.java
@@ -54,6 +54,7 @@ public final class TaskSMD extends StateModelDefinition {
     builder.addState(TaskPartitionState.COMPLETED.name(), 3);
     builder.addState(TaskPartitionState.TIMED_OUT.name(), 4);
     builder.addState(TaskPartitionState.TASK_ERROR.name(), 5);
+    builder.addState(TaskPartitionState.TASK_ABORTED.name(), 6);
     builder.addState(TaskPartitionState.DROPPED.name());
 
     // add transitions
@@ -62,22 +63,26 @@ public final class TaskSMD extends StateModelDefinition {
     builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.COMPLETED.name(), 2);
     builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.TIMED_OUT.name(), 3);
     builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.TASK_ERROR.name(), 4);
-    builder.addTransition(TaskPartitionState.STOPPED.name(), TaskPartitionState.RUNNING.name(), 5);
+    builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.TASK_ABORTED.name(), 5);
+    builder.addTransition(TaskPartitionState.STOPPED.name(), TaskPartitionState.RUNNING.name(), 6);
 
     // All states have a transition to DROPPED.
-    builder.addTransition(TaskPartitionState.INIT.name(), TaskPartitionState.DROPPED.name(), 6);
-    builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.DROPPED.name(), 7);
-    builder.addTransition(TaskPartitionState.COMPLETED.name(), TaskPartitionState.DROPPED.name(), 8);
-    builder.addTransition(TaskPartitionState.STOPPED.name(), TaskPartitionState.DROPPED.name(), 9);
-    builder.addTransition(TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.DROPPED.name(), 10);
-    builder.addTransition(TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.DROPPED.name(), 11);
+    builder.addTransition(TaskPartitionState.INIT.name(), TaskPartitionState.DROPPED.name(), 7);
+    builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.DROPPED.name(), 8);
+    builder.addTransition(TaskPartitionState.COMPLETED.name(), TaskPartitionState.DROPPED.name(), 9);
+    builder.addTransition(TaskPartitionState.STOPPED.name(), TaskPartitionState.DROPPED.name(), 10);
+    builder.addTransition(TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.DROPPED.name(), 11);
+    builder.addTransition(TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.DROPPED.name(), 12);
+    builder.addTransition(TaskPartitionState.TASK_ABORTED.name(), TaskPartitionState.DROPPED.name(), 13);
+
 
     // All states, except DROPPED, have a transition to INIT.
-    builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.INIT.name(), 12);
-    builder.addTransition(TaskPartitionState.COMPLETED.name(), TaskPartitionState.INIT.name(), 13);
-    builder.addTransition(TaskPartitionState.STOPPED.name(), TaskPartitionState.INIT.name(), 14);
-    builder.addTransition(TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.INIT.name(), 15);
-    builder.addTransition(TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.INIT.name(), 16);
+    builder.addTransition(TaskPartitionState.RUNNING.name(), TaskPartitionState.INIT.name(), 14);
+    builder.addTransition(TaskPartitionState.COMPLETED.name(), TaskPartitionState.INIT.name(), 15);
+    builder.addTransition(TaskPartitionState.STOPPED.name(), TaskPartitionState.INIT.name(), 16);
+    builder.addTransition(TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.INIT.name(), 17);
+    builder.addTransition(TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.INIT.name(), 18);
+    builder.addTransition(TaskPartitionState.TASK_ABORTED.name(), TaskPartitionState.INIT.name(), 19);
 
     return builder.build();
   }
@@ -99,6 +104,7 @@ public final class TaskSMD extends StateModelDefinition {
     statePriorityList.add(TaskPartitionState.COMPLETED.name());
     statePriorityList.add(TaskPartitionState.TIMED_OUT.name());
     statePriorityList.add(TaskPartitionState.TASK_ERROR.name());
+    statePriorityList.add(TaskPartitionState.TASK_ABORTED.name());
     statePriorityList.add(TaskPartitionState.DROPPED.name());
     record.setListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString(), statePriorityList);
     for (String state : statePriorityList) {
@@ -115,6 +121,7 @@ public final class TaskSMD extends StateModelDefinition {
     states.add(TaskPartitionState.COMPLETED.name());
     states.add(TaskPartitionState.TIMED_OUT.name());
     states.add(TaskPartitionState.TASK_ERROR.name());
+    states.add(TaskPartitionState.TASK_ABORTED.name());
     states.add(TaskPartitionState.DROPPED.name());
 
     List<Transition> transitions = new ArrayList<Transition>();
@@ -123,6 +130,7 @@ public final class TaskSMD extends StateModelDefinition {
     transitions.add(new Transition(TaskPartitionState.RUNNING.name(), TaskPartitionState.COMPLETED.name()));
     transitions.add(new Transition(TaskPartitionState.RUNNING.name(), TaskPartitionState.TIMED_OUT.name()));
     transitions.add(new Transition(TaskPartitionState.RUNNING.name(), TaskPartitionState.TASK_ERROR.name()));
+    transitions.add(new Transition(TaskPartitionState.RUNNING.name(), TaskPartitionState.TASK_ABORTED.name()));
     transitions.add(new Transition(TaskPartitionState.STOPPED.name(), TaskPartitionState.RUNNING.name()));
 
     // All states have a transition to DROPPED.
@@ -132,6 +140,8 @@ public final class TaskSMD extends StateModelDefinition {
     transitions.add(new Transition(TaskPartitionState.STOPPED.name(), TaskPartitionState.DROPPED.name()));
     transitions.add(new Transition(TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.DROPPED.name()));
     transitions.add(new Transition(TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.DROPPED.name()));
+    transitions.add(new Transition(TaskPartitionState.TASK_ABORTED.name(), TaskPartitionState.DROPPED.name()));
+
 
     // All states, except DROPPED, have a transition to INIT.
     transitions.add(new Transition(TaskPartitionState.RUNNING.name(), TaskPartitionState.INIT.name()));
@@ -139,6 +149,7 @@ public final class TaskSMD extends StateModelDefinition {
     transitions.add(new Transition(TaskPartitionState.STOPPED.name(), TaskPartitionState.INIT.name()));
     transitions.add(new Transition(TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.INIT.name()));
     transitions.add(new Transition(TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.INIT.name()));
+    transitions.add(new Transition(TaskPartitionState.TASK_ABORTED.name(), TaskPartitionState.INIT.name()));
 
     StateTransitionTableBuilder builder = new StateTransitionTableBuilder();
     Map<String, Map<String, String>> next = builder.buildTransitionTable(states, transitions);
@@ -154,6 +165,7 @@ public final class TaskSMD extends StateModelDefinition {
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.RUNNING.name(), TaskPartitionState.COMPLETED.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.RUNNING.name(), TaskPartitionState.TIMED_OUT.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.RUNNING.name(), TaskPartitionState.TASK_ERROR.name()));
+    stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.RUNNING.name(), TaskPartitionState.TASK_ABORTED.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.STOPPED.name(), TaskPartitionState.RUNNING.name()));
 
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.INIT.name(), TaskPartitionState.DROPPED.name()));
@@ -162,12 +174,14 @@ public final class TaskSMD extends StateModelDefinition {
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.STOPPED.name(), TaskPartitionState.DROPPED.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.DROPPED.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.DROPPED.name()));
+    stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.TASK_ABORTED.name(), TaskPartitionState.DROPPED.name()));
 
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.RUNNING.name(), TaskPartitionState.INIT.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.COMPLETED.name(), TaskPartitionState.INIT.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.STOPPED.name(), TaskPartitionState.INIT.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.TIMED_OUT.name(), TaskPartitionState.INIT.name()));
     stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.TASK_ERROR.name(), TaskPartitionState.INIT.name()));
+    stateTransitionPriorityList.add(String.format("%s-%s", TaskPartitionState.TASK_ABORTED.name(), TaskPartitionState.INIT.name()));
 
     record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(),
                         stateTransitionPriorityList);

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
index 1eeca60..d26c83b 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobConfig.java
@@ -99,7 +99,6 @@ public class JobConfig {
      * The amount of time in ms to wait before retrying a task
      */
     TaskRetryDelay,
-
     /**
      * Whether failure of directly dependent jobs should fail this job.
      */

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index 5b41773..b02089f 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -314,15 +314,17 @@ public class JobRebalancer extends TaskRebalancer {
           break;
         case TIMED_OUT:
         case TASK_ERROR:
+        case TASK_ABORTED:
         case ERROR: {
           donePartitions.add(pId); // The task may be rescheduled on a different instance.
           LOG.debug(String.format(
-              "Task partition %s has error state %s. Marking as such in rebalancer context.",
-              pName, currState));
+              "Task partition %s has error state %s. Marking as such in rebalancer context.", pName,
+              currState));
           markPartitionError(jobCtx, pId, currState, true);
           // The error policy is to fail the task as soon a single partition fails for a specified
-          // maximum number of attempts.
-          if (jobCtx.getPartitionNumAttempts(pId) >= jobCfg.getMaxAttemptsPerTask()) {
+          // maximum number of attempts or task is in ABORTED state.
+          if (jobCtx.getPartitionNumAttempts(pId) >= jobCfg.getMaxAttemptsPerTask() ||
+              currState.equals(TaskPartitionState.TASK_ABORTED)) {
             // If the user does not require this task to succeed in order for the job to succeed,
             // then we don't have to fail the job right now
             boolean successOptional = false;
@@ -352,6 +354,8 @@ public class JobRebalancer extends TaskRebalancer {
               skippedPartitions.add(pId);
               partitionsToDropFromIs.add(pId);
             }
+
+            LOG.debug("skippedPartitions:" + skippedPartitions);
           } else {
             // Mark the task to be started at some later time (if enabled)
             markPartitionDelayed(jobCfg, jobCtx, pId);
@@ -391,7 +395,7 @@ public class JobRebalancer extends TaskRebalancer {
       // any new assignments.
       // This includes all completed, failed, delayed, and already assigned partitions.
       Set<Integer> excludeSet = Sets.newTreeSet(assignedPartitions);
-      addCompletedPartitions(excludeSet, jobCtx, allPartitions);
+      addCompletedTasks(excludeSet, jobCtx, allPartitions);
       addGiveupPartitions(excludeSet, jobCtx, allPartitions, jobCfg);
       excludeSet.addAll(skippedPartitions);
       excludeSet.addAll(getNonReadyPartitions(jobCtx, currentTime));
@@ -529,7 +533,7 @@ public class JobRebalancer extends TaskRebalancer {
     }
   }
 
-  private static void addCompletedPartitions(Set<Integer> set, JobContext ctx,
+  private static void addCompletedTasks(Set<Integer> set, JobContext ctx,
       Iterable<Integer> pIds) {
     for (Integer pId : pIds) {
       TaskPartitionState state = ctx.getPartitionState(pId);
@@ -540,6 +544,11 @@ public class JobRebalancer extends TaskRebalancer {
   }
 
   private static boolean isTaskGivenup(JobContext ctx, JobConfig cfg, int pId) {
+    TaskPartitionState state = ctx.getPartitionState(pId);
+    if (state != null && (state.equals(TaskPartitionState.TASK_ABORTED) | state
+        .equals(TaskPartitionState.ERROR))) {
+      return true;
+    }
     return ctx.getPartitionNumAttempts(pId) >= cfg.getMaxAttemptsPerTask();
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/TaskPartitionState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskPartitionState.java b/helix-core/src/main/java/org/apache/helix/task/TaskPartitionState.java
index d41668d..5b4a5a4 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskPartitionState.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskPartitionState.java
@@ -33,8 +33,10 @@ public enum TaskPartitionState {
   COMPLETED,
   /** Indicates that the task timed out. */
   TIMED_OUT,
-  /** Indicates an error occurred during task execution. */
+  /** Indicates an error occurred during task execution, but the task can be retried. */
   TASK_ERROR,
+  /** Indicates an error occurred during task execution, and the task should not be retried. */
+  TASK_ABORTED,
   /** Helix's own internal error state. */
   ERROR,
   /** A Helix internal state. */

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/TaskResult.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskResult.java b/helix-core/src/main/java/org/apache/helix/task/TaskResult.java
index 95b8d72..02cd162 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskResult.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskResult.java
@@ -34,8 +34,14 @@ public class TaskResult {
      * called.
      */
     CANCELED,
-    /** The task encountered an error from which it could not recover. */
-    ERROR
+    /** The task encountered an error from which it can not recover.
+     * This is equivalent to {@link org.apache.helix.task.TaskResult.Status#FAILED}.*/
+    @Deprecated
+    ERROR,
+    /** The task encountered an error which can not be recovered from this run, but it may still succeed by retrying the task. */
+    FAILED,
+    /** The task encountered an error, which will not be recoverable even with retrying the task */
+    FATAL_FAILED
   }
 
   private final Status _status;

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java b/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
index 1bf88ec..c43d0ce 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskRunner.java
@@ -71,8 +71,8 @@ public class TaskRunner implements Runnable {
       } catch (ThreadDeath death) {
         throw death;
       } catch (Throwable t) {
-        LOG.error("Problem running the task", t);
-        _result = new TaskResult(Status.ERROR, null);
+        LOG.error("Problem running the task, report task as FAILED.", t);
+        _result = new TaskResult(Status.FAILED, null);
       }
 
       switch (_result.getStatus()) {
@@ -88,8 +88,14 @@ public class TaskRunner implements Runnable {
       case ERROR:
         requestStateTransition(TaskPartitionState.TASK_ERROR);
         break;
+      case FAILED:
+        requestStateTransition(TaskPartitionState.TASK_ERROR);
+        break;
+      case FATAL_FAILED:
+        requestStateTransition(TaskPartitionState.TASK_ABORTED);
+        break;
       default:
-        throw new AssertionError("Unknown result type.");
+        throw new AssertionError("Unknown task result type: " + _result.getStatus().name());
       }
     } catch (Exception e) {
       requestStateTransition(TaskPartitionState.TASK_ERROR);

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java b/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
index d3ee003..ba68a78 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskStateModel.java
@@ -133,7 +133,27 @@ public class TaskStateModel extends StateModel {
     }
 
     TaskResult r = _taskRunner.waitTillDone();
-    if (r.getStatus() != TaskResult.Status.ERROR) {
+    if (r.getStatus() != TaskResult.Status.ERROR && r.getStatus() != TaskResult.Status.FAILED) {
+      throw new IllegalStateException(String.format(
+          "Partition %s received a state transition to %s but the result status code is %s.",
+          msg.getPartitionName(), msg.getToState(), r.getStatus()));
+    }
+
+    timeout_task.cancel(false);
+
+    return r.getInfo();
+  }
+
+  @Transition(to = "TASK_ABORTED", from = "RUNNING")
+  public String onBecomeTaskAbortedFromRunning(Message msg, NotificationContext context) {
+    String taskPartition = msg.getPartitionName();
+    if (_taskRunner == null) {
+      throw new IllegalStateException(String.format(
+          "Invalid state transition. There is no running task for partition %s.", taskPartition));
+    }
+
+    TaskResult r = _taskRunner.waitTillDone();
+    if (r.getStatus() != TaskResult.Status.FATAL_FAILED) {
       throw new IllegalStateException(String.format(
           "Partition %s received a state transition to %s but the result status code is %s.",
           msg.getPartitionName(), msg.getToState(), r.getStatus()));
@@ -189,6 +209,11 @@ public class TaskStateModel extends StateModel {
     reset();
   }
 
+  @Transition(to = "DROPPED", from = "TASK_ABORTED")
+  public void onBecomeDroppedFromTaskAborted(Message msg, NotificationContext context) {
+    reset();
+  }
+
   @Transition(to = "INIT", from = "RUNNING")
   public void onBecomeInitFromRunning(Message msg, NotificationContext context) {
     String taskPartition = msg.getPartitionName();
@@ -223,6 +248,11 @@ public class TaskStateModel extends StateModel {
     reset();
   }
 
+  @Transition(to = "INIT", from = "TASK_ABORTED")
+  public void onBecomeInitFromTaskAborted(Message msg, NotificationContext context) {
+    reset();
+  }
+
   @Override
   public void reset() {
     if (_taskRunner != null) {

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
index ddd37d5..2881b61 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java
@@ -35,7 +35,7 @@ import org.apache.log4j.Logger;
  * Provides a typed interface to workflow level configurations. Validates the configurations.
  */
 // TODO: extends WorkflowConfig from ResourceConfig
-public class WorkflowConfig {
+public class  WorkflowConfig {
   private static final Logger LOG = Logger.getLogger(WorkflowConfig.class);
 
   /**

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
index dad9949..f415b8e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/MockTask.java
@@ -19,27 +19,45 @@ package org.apache.helix.integration.task;
  * under the License.
  */
 
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.Task;
 import org.apache.helix.task.TaskCallbackContext;
+import org.apache.helix.task.TaskConfig;
 import org.apache.helix.task.TaskResult;
 
 public class MockTask implements Task {
   public static final String TASK_COMMAND = "Reindex";
-  private static final String TIMEOUT_CONFIG = "Timeout";
+  public static final String TIMEOUT_CONFIG = "Timeout";
+  public static final String TASK_RESULT_STATUS = "TaskResultStatus";
+  public static final String THROW_EXCEPTION = "ThrowException";
   private final long _delay;
   private volatile boolean _canceled;
+  private TaskResult.Status _taskResultStatus;
+  private boolean _throwException;
 
   public MockTask(TaskCallbackContext context) {
-    JobConfig jobCfg = context.getJobConfig();
-    Map<String, String> cfg = jobCfg.getJobCommandConfigMap();
+    Map<String, String> cfg = context.getJobConfig().getJobCommandConfigMap();
     if (cfg == null) {
-      cfg = Collections.emptyMap();
+      cfg = new HashMap<String, String>();
     }
+
+    TaskConfig taskConfig = context.getTaskConfig();
+    Map<String, String> taskCfg = taskConfig.getConfigMap();
+    if (taskCfg != null) {
+      cfg.putAll(taskCfg);
+    }
+
     _delay = cfg.containsKey(TIMEOUT_CONFIG) ? Long.parseLong(cfg.get(TIMEOUT_CONFIG)) : 100L;
+    _taskResultStatus = cfg.containsKey(TASK_RESULT_STATUS) ?
+        TaskResult.Status.valueOf(cfg.get(TASK_RESULT_STATUS)) :
+        TaskResult.Status.COMPLETED;
+    _throwException = cfg.containsKey(THROW_EXCEPTION) ?
+        Boolean.valueOf(cfg.containsKey(THROW_EXCEPTION)) :
+        false;
   }
 
   @Override
@@ -55,7 +73,12 @@ public class MockTask implements Task {
       sleep(50);
     }
     timeLeft = expiry - System.currentTimeMillis();
-    return new TaskResult(TaskResult.Status.COMPLETED, String.valueOf(timeLeft < 0 ? 0 : timeLeft));
+
+    if (_throwException) {
+      throw new RuntimeException("Test failed");
+    }
+
+    return new TaskResult(_taskResultStatus, String.valueOf(timeLeft < 0 ? 0 : timeLeft));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
index 8262b9b..65ec458 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRecurringJobQueue.java
@@ -42,7 +42,6 @@ import com.google.common.collect.Sets;
 
 public class TestRecurringJobQueue extends TaskTestBase {
   private static final Logger LOG = Logger.getLogger(TestRecurringJobQueue.class);
-  private static final String TIMEOUT_CONFIG = "Timeout";
 
   @Test
   public void deleteRecreateRecurrentQueue() throws Exception {
@@ -120,7 +119,7 @@ public class TestRecurringJobQueue extends TaskTestBase {
 
     // Create and Enqueue jobs
     List<String> currentJobNames = new ArrayList<String>();
-    Map<String, String> commandConfig = ImmutableMap.of(TIMEOUT_CONFIG, String.valueOf(500));
+    Map<String, String> commandConfig = ImmutableMap.of(MockTask.TIMEOUT_CONFIG, String.valueOf(500));
     Thread.sleep(100);
     for (int i = 0; i <= 4; i++) {
       String targetPartition = (i == 0) ? "MASTER" : "SLAVE";
@@ -211,7 +210,7 @@ public class TestRecurringJobQueue extends TaskTestBase {
     // create jobs
     List<JobConfig.Builder> jobs = new ArrayList<JobConfig.Builder>();
     List<String> jobNames = new ArrayList<String>();
-    Map<String, String> commandConfig = ImmutableMap.of(TIMEOUT_CONFIG, String.valueOf(500));
+    Map<String, String> commandConfig = ImmutableMap.of(MockTask.TIMEOUT_CONFIG, String.valueOf(500));
 
     final int JOB_COUNTS = 3;
     for (int i = 0; i < JOB_COUNTS; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/7f184839/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
new file mode 100644
index 0000000..5fa370d
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskConditionalRetry.java
@@ -0,0 +1,102 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.TaskConfig;
+import org.apache.helix.task.TaskPartitionState;
+import org.apache.helix.task.TaskResult;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/**
+ * Test Conditional Task Retry
+ */
+public class TestTaskConditionalRetry extends TaskTestBase {
+
+  @Test public void test() throws Exception {
+    int taskRetryCount = 5;
+    int num_tasks = 5;
+
+    String jobResource = TestHelper.getTestMethodName();
+    JobConfig.Builder jobBuilder = new JobConfig.Builder();
+    jobBuilder.setCommand(MockTask.TASK_COMMAND).setTimeoutPerTask(10000)
+        .setMaxAttemptsPerTask(taskRetryCount).setFailureThreshold(Integer.MAX_VALUE);
+
+    // create each task configs.
+    final int abortedTask = 1;
+    final int failedTask = 2;
+    final int exceptionTask = 3;
+
+    List<TaskConfig> taskConfigs = new ArrayList<TaskConfig>();
+    for (int j = 0; j < num_tasks; j++) {
+      TaskConfig.Builder configBuilder = new TaskConfig.Builder().setTaskId("task_" + j);
+      switch (j) {
+      case abortedTask:
+        configBuilder.addConfig(MockTask.TASK_RESULT_STATUS, TaskResult.Status.FATAL_FAILED.name());
+        break;
+      case failedTask:
+        configBuilder.addConfig(MockTask.TASK_RESULT_STATUS, TaskResult.Status.FAILED.name());
+        break;
+      case exceptionTask:
+        configBuilder.addConfig(MockTask.THROW_EXCEPTION, Boolean.TRUE.toString());
+        break;
+      default:
+        break;
+      }
+      configBuilder.setTargetPartition(String.valueOf(j));
+      taskConfigs.add(configBuilder.build());
+    }
+    jobBuilder.addTaskConfigs(taskConfigs);
+
+    Workflow flow =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
+
+    _driver.start(flow);
+
+    // Wait until the job completes.
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED);
+
+    JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
+    for (int i = 0; i < num_tasks; i++) {
+      TaskPartitionState state = ctx.getPartitionState(i);
+      int retriedCount = ctx.getPartitionNumAttempts(i);
+      String taskId = ctx.getTaskIdForPartition(i);
+
+      if (taskId.equals("task_" + abortedTask)) {
+        Assert.assertEquals(state, TaskPartitionState.TASK_ABORTED);
+        Assert.assertEquals(retriedCount, 1);
+      } else if (taskId.equals("task_" + failedTask) || taskId.equals("task_" + exceptionTask)) {
+        Assert.assertEquals(state, TaskPartitionState.TASK_ERROR);
+        Assert.assertEquals(retriedCount, taskRetryCount);
+      } else {
+        Assert.assertEquals(state, TaskPartitionState.COMPLETED);
+        Assert.assertEquals(retriedCount, 1);
+      }
+    }
+  }
+}


[12/33] helix git commit: Add static methods into TaskDriver for getting configuration/context for jobs and workflows.

Posted by lx...@apache.org.
Add static methods into TaskDriver for getting configuration/context for jobs and workflows.


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

Branch: refs/heads/helix-0.6.x
Commit: aeb6f3ec7ab973316fd3468e9cbb0052a6a4306e
Parents: d386aff
Author: Lei Xia <lx...@linkedin.com>
Authored: Mon Mar 21 13:46:01 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 14:53:34 2016 -0700

----------------------------------------------------------------------
 .../webapp/resources/JobQueueResource.java      | 19 +++++++------------
 .../helix/webapp/resources/JobResource.java     | 11 ++---------
 .../java/org/apache/helix/task/TaskDriver.java  | 16 ++++++++++++++++
 .../java/org/apache/helix/task/TaskUtil.java    | 20 ++++++++++----------
 .../integration/task/TestTaskRebalancer.java    |  2 --
 5 files changed, 35 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/aeb6f3ec/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueueResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueueResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueueResource.java
index 830e16b..954ae73 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueueResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobQueueResource.java
@@ -21,15 +21,10 @@ package org.apache.helix.webapp.resources;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
-import org.apache.helix.HelixProperty;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyPathConfig;
-import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.store.HelixPropertyStore;
-import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskUtil;
@@ -91,14 +86,14 @@ public class JobQueueResource extends ServerResource {
         ClusterRepresentationUtil.getClusterDataAccessor(zkClient, clusterName);
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
 
-    // Get job queue config
-    HelixProperty jobQueueConfig = accessor.getProperty(keyBuilder.resourceConfig(jobQueueName));
+    TaskDriver taskDriver = new TaskDriver(zkClient, clusterName);
 
+    // Get job queue config
+    // TODO: fix this to use workflowConfig.
+    ResourceConfig jobQueueConfig = accessor.getProperty(keyBuilder.resourceConfig(jobQueueName));
+    
     // Get job queue context
-    String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, clusterName);
-    HelixPropertyStore<ZNRecord> propertyStore =
-        new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(zkClient), path, null);
-    WorkflowContext ctx = TaskUtil.getWorkflowContext(propertyStore, jobQueueName);
+    WorkflowContext ctx = taskDriver.getWorkflowContext(jobQueueName);
 
     // Create the result
     ZNRecord hostedEntitiesRecord = new ZNRecord(jobQueueName);

http://git-wip-us.apache.org/repos/asf/helix/blob/aeb6f3ec/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobResource.java
index cdcde35..d31c81b 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/JobResource.java
@@ -25,10 +25,7 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.store.HelixPropertyStore;
-import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.helix.task.JobContext;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskUtil;
@@ -119,14 +116,10 @@ public class JobResource extends ServerResource {
     // Get job queue config
     String namespacedJobName = TaskUtil.getNamespacedJobName(jobQueueName, jobName);
     HelixProperty jobConfig = accessor.getProperty(keyBuilder.resourceConfig(namespacedJobName));
+    TaskDriver taskDriver = new TaskDriver(zkClient, clusterName);
 
     // Get job queue context
-    JobContext ctx = null;
-    String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, clusterName);
-    HelixPropertyStore<ZNRecord> propertyStore =
-        new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(zkClient), path, null);
-
-    ctx = TaskUtil.getJobContext(propertyStore, namespacedJobName);
+    JobContext ctx = taskDriver.getJobContext(namespacedJobName);
 
     // Create the result
     ZNRecord hostedEntitiesRecord = new ZNRecord(namespacedJobName);

http://git-wip-us.apache.org/repos/asf/helix/blob/aeb6f3ec/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index 99bcb62..c0d7852 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -733,6 +733,22 @@ public class TaskDriver {
     return TaskUtil.getJobContext(_propertyStore, job);
   }
 
+  public static JobContext getJobContext(HelixManager manager, String job) {
+    return TaskUtil.getJobContext(manager, job);
+  }
+
+  public static WorkflowConfig getWorkflowConfig(HelixManager manager, String workflow) {
+    return TaskUtil.getWorkflowCfg(manager, workflow);
+  }
+
+  public static WorkflowContext getWorkflowContext(HelixManager manager, String workflow) {
+    return TaskUtil.getWorkflowContext(manager, workflow);
+  }
+
+  public static JobConfig getJobConfig(HelixManager manager, String job) {
+    return TaskUtil.getJobCfg(manager, job);
+  }
+
   public void list(String resource) {
     WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_accessor, resource);
     if (wCfg == null) {

http://git-wip-us.apache.org/repos/asf/helix/blob/aeb6f3ec/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
index 513c14e..8745a82 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java
@@ -51,7 +51,7 @@ public class TaskUtil {
 
   /**
    * Parses job resource configurations in Helix into a {@link JobConfig} object.
-   * This method is internal API, please use TaskDriver.getJobConfig();
+   * This method is internal API, please use the corresponding one in TaskDriver.getJobConfig();
    *
    * @param accessor    Accessor to access Helix configs
    * @param jobResource The name of the job resource
@@ -78,7 +78,7 @@ public class TaskUtil {
 
   /**
    * Parses job resource configurations in Helix into a {@link JobConfig} object.
-   * This method is internal API, please use TaskDriver.getJobConfig();
+   * This method is internal API, please use the corresponding one in TaskDriver.getJobConfig();
    *
    * @param manager     HelixManager object used to connect to Helix.
    * @param jobResource The name of the job resource.
@@ -91,7 +91,7 @@ public class TaskUtil {
 
   /**
    * Parses workflow resource configurations in Helix into a {@link WorkflowConfig} object.
-   * This method is internal API, please use TaskDriver.getWorkflowConfig();
+   * This method is internal API, please use the corresponding one in TaskDriver.getWorkflowConfig();
    *
    * @param accessor  Accessor to access Helix configs
    * @param workflow The name of the workflow.
@@ -112,7 +112,7 @@ public class TaskUtil {
 
   /**
    * Parses workflow resource configurations in Helix into a {@link WorkflowConfig} object.
-   * This method is internal API, please use TaskDriver.getWorkflowConfig();
+   * This method is internal API, please use the corresponding one in TaskDriver.getWorkflowConfig();
    *
    * @param manager          Helix manager object used to connect to Helix.
    * @param workflow The name of the workflow resource.
@@ -143,7 +143,7 @@ public class TaskUtil {
    * @param jobResource   The name of the job
    * @return the {@link JobContext}, or null if none is available
    */
-  public static JobContext getJobContext(HelixPropertyStore<ZNRecord> propertyStore,
+  protected static JobContext getJobContext(HelixPropertyStore<ZNRecord> propertyStore,
       String jobResource) {
     ZNRecord r = propertyStore
         .get(Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, jobResource, CONTEXT_NODE),
@@ -179,13 +179,13 @@ public class TaskUtil {
 
   /**
    * Get the runtime context of a single workflow.
-   * This method is internal API, please use TaskDriver.getWorkflowContext();
+   * This method is internal API, please use the corresponding one in TaskDriver.getWorkflowContext();
    *
    * @param propertyStore    Property store of the cluster
    * @param workflowResource The name of the workflow
    * @return the {@link WorkflowContext}, or null if none is available
    */
-  public static WorkflowContext getWorkflowContext(HelixPropertyStore<ZNRecord> propertyStore,
+  protected static WorkflowContext getWorkflowContext(HelixPropertyStore<ZNRecord> propertyStore,
       String workflowResource) {
     ZNRecord r = propertyStore.get(
         Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, workflowResource, CONTEXT_NODE),
@@ -195,7 +195,7 @@ public class TaskUtil {
 
   /**
    * Get the runtime context of a single workflow.
-   * This method is internal API, please use TaskDriver.getWorkflowContext();
+   * This method is internal API, please use the corresponding one in TaskDriver.getWorkflowContext();
    *
    * @param manager          a connection to Helix
    * @param workflowResource the name of the workflow
@@ -212,7 +212,7 @@ public class TaskUtil {
    * @param workflowResource the name of the workflow
    * @param ctx              the up-to-date {@link WorkflowContext} for the workflow
    */
-  public static void setWorkflowContext(HelixManager manager, String workflowResource,
+  protected static void setWorkflowContext(HelixManager manager, String workflowResource,
       WorkflowContext ctx) {
     manager.getHelixPropertyStore().set(
         Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, workflowResource, CONTEXT_NODE),
@@ -298,7 +298,7 @@ public class TaskUtil {
    * @param accessor Helix data accessor
    * @param resource the name of the resource changed to triggering the execution
    */
-  public static void invokeRebalance(HelixDataAccessor accessor, String resource) {
+  protected static void invokeRebalance(HelixDataAccessor accessor, String resource) {
     // The pipeline is idempotent, so touching an ideal state is enough to trigger a pipeline run
     LOG.info("invoke rebalance for " + resource);
     PropertyKey key = accessor.keyBuilder().idealStates(resource);

http://git-wip-us.apache.org/repos/asf/helix/blob/aeb6f3ec/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index 3a5b179..74b10fd 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -184,8 +184,6 @@ public class TestTaskRebalancer extends ZkIntegrationTestBase {
     // Wait for job to finish and expire
     TaskTestUtil.pollForWorkflowState(_driver, jobName, TaskState.COMPLETED);
     Thread.sleep(expiry);
-    TaskUtil.invokeRebalance(_manager.getHelixDataAccessor(), flow.getName());
-    Thread.sleep(expiry);
 
     // Ensure workflow config and context were cleaned up by now
     Assert.assertFalse(_manager.getHelixPropertyStore().exists(workflowPropStoreKey,


[28/33] helix git commit: Check whether instance is disable when assigning tasks to instances in TaskRebalancer.

Posted by lx...@apache.org.
Check whether instance is disable when assigning tasks to instances in TaskRebalancer.


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

Branch: refs/heads/helix-0.6.x
Commit: c3624e082b4025945f1d7755c73ce999bcac6783
Parents: 7e9041f
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu May 5 15:51:58 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:18:25 2016 -0700

----------------------------------------------------------------------
 .../FixedTargetTaskAssignmentCalculator.java    | 18 +++++-
 .../helix/integration/task/TaskTestBase.java    | 15 ++++-
 .../task/TestTaskWithInstanceDisabled.java      | 58 ++++++++++++++++++++
 3 files changed, 88 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/c3624e08/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
index 60cd92f..0a2e8c5 100644
--- a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
+++ b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskAssignmentCalculator.java
@@ -31,6 +31,7 @@ import java.util.TreeSet;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.ResourceAssignment;
@@ -65,7 +66,7 @@ public class FixedTargetTaskAssignmentCalculator extends TaskAssignmentCalculato
     }
     Set<String> tgtStates = jobCfg.getTargetPartitionStates();
     return getTgtPartitionAssignment(currStateOutput, instances, tgtIs, tgtStates, partitionSet,
-        jobContext);
+        jobContext, cache);
   }
 
   /**
@@ -130,7 +131,7 @@ public class FixedTargetTaskAssignmentCalculator extends TaskAssignmentCalculato
    */
   private static Map<String, SortedSet<Integer>> getTgtPartitionAssignment(
       CurrentStateOutput currStateOutput, Iterable<String> instances, IdealState tgtIs,
-      Set<String> tgtStates, Set<Integer> includeSet, JobContext jobCtx) {
+      Set<String> tgtStates, Set<Integer> includeSet, JobContext jobCtx, ClusterDataCache cache) {
     Map<String, SortedSet<Integer>> result = new HashMap<String, SortedSet<Integer>>();
     for (String instance : instances) {
       result.put(instance, new TreeSet<Integer>());
@@ -151,6 +152,19 @@ public class FixedTargetTaskAssignmentCalculator extends TaskAssignmentCalculato
           if (pendingMessage != null) {
             continue;
           }
+
+          InstanceConfig instanceConfig = cache.getInstanceConfigMap().get(instance);
+
+          if (instanceConfig == null) {
+            LOG.error("Instance config not found for instance : " + instance);
+            continue;
+          }
+
+          if (!instanceConfig.getInstanceEnabled()) {
+            LOG.debug("Instance has been disabled, ignore instance : " + instance);
+            continue;
+          }
+
           String s =
               currStateOutput.getCurrentState(tgtIs.getResourceName(), new Partition(pName),
                   instance);

http://git-wip-us.apache.org/repos/asf/helix/blob/c3624e08/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
index d6cf9bd..1bb72cc 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
@@ -51,6 +51,7 @@ public class TaskTestBase extends ZkIntegrationTestBase {
   protected int _numDbs = 1;
 
   protected Boolean _partitionVary = true;
+  protected Boolean _instanceGroupTag = false;
 
   protected ClusterControllerManager _controller;
 
@@ -76,6 +77,9 @@ public class TaskTestBase extends ZkIntegrationTestBase {
     for (int i = 0; i < _numNodes; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
       _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      if (_instanceGroupTag) {
+        _setupTool.addInstanceTag(CLUSTER_NAME, storageNodeName, "TESTTAG" + i);
+      }
     }
 
     // Set up target db
@@ -90,7 +94,16 @@ public class TaskTestBase extends ZkIntegrationTestBase {
         _testDbs.add(db);
       }
     } else {
-      _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions, MASTER_SLAVE_STATE_MODEL);
+      if (_instanceGroupTag) {
+        _setupTool
+            .addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions,
+                "OnlineOffline", IdealState.RebalanceMode.FULL_AUTO.name());
+        IdealState idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+        idealState.setInstanceGroupTag("TESTTAG0");
+        _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, idealState);
+      } else {
+        _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions, MASTER_SLAVE_STATE_MODEL);
+      }
       _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numReplicas);
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c3624e08/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
new file mode 100644
index 0000000..1c5bd36
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
@@ -0,0 +1,58 @@
+package org.apache.helix.integration.task;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.TestHelper;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.TaskState;
+import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.Workflow;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestTaskWithInstanceDisabled extends TaskTestBase {
+  @Override
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numDbs = 1;
+    _numNodes = 2;
+    _numParitions = 1;
+    _numReplicas = 1;
+    _partitionVary = false;
+    super.beforeClass();
+  }
+  @Test
+  public void testTaskWithInstanceDisabled() throws InterruptedException {
+    _setupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, PARTICIPANT_PREFIX + "_" + (_startPort + 0), false);
+    String jobResource = TestHelper.getTestMethodName();
+    JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)
+        .setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB);
+    Workflow flow =
+        WorkflowGenerator.generateSingleJobWorkflowBuilder(jobResource, jobBuilder).build();
+    _driver.start(flow);
+
+    TaskTestUtil.pollForWorkflowState(_driver, jobResource, TaskState.COMPLETED, TaskState.FAILED);
+    JobContext ctx = _driver.getJobContext(TaskUtil.getNamespacedJobName(jobResource));
+    Assert.assertEquals(ctx.getAssignedParticipant(0), PARTICIPANT_PREFIX + "_" + (_startPort + 1));
+  }
+}


[24/33] helix git commit: Add Partition task start time

Posted by lx...@apache.org.
Add Partition task start time

Add setup and get start time for partition tasks.

RB=715086
G=nuage-reviewers
R=lxia,cji
A=lxia


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

Branch: refs/heads/helix-0.6.x
Commit: ddefebb245f4f43c8c64a9f8cc1a2536ae25eab4
Parents: 2efa448
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Apr 28 11:36:20 2016 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Tue Jul 5 16:17:38 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobContext.java  | 23 +++++++++++++++++---
 .../org/apache/helix/task/JobRebalancer.java    |  1 +
 .../org/apache/helix/task/WorkflowContext.java  |  1 +
 3 files changed, 22 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ddefebb2/helix-core/src/main/java/org/apache/helix/task/JobContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobContext.java b/helix-core/src/main/java/org/apache/helix/task/JobContext.java
index 77885cd..2057f27 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobContext.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobContext.java
@@ -59,7 +59,7 @@ public class JobContext extends HelixProperty {
   public long getStartTime() {
     String tStr = _record.getSimpleField(ContextProperties.START_TIME.toString());
     if (tStr == null) {
-      return -1;
+      return WorkflowContext.UNSTARTED;
     }
     return Long.parseLong(tStr);
   }
@@ -121,6 +121,23 @@ public class JobContext extends HelixProperty {
     return Integer.parseInt(nStr);
   }
 
+  public void setPartitionStartTime(int p, long t) {
+    Map<String, String> map = getMapField(p);
+    map.put(ContextProperties.START_TIME.toString(), String.valueOf(t));
+  }
+
+  public long getPartitionStartTime(int p) {
+    Map<String, String> map = getMapField(p);
+    if (map == null) {
+      return WorkflowContext.UNSTARTED;
+    }
+    String tStr = map.get(ContextProperties.START_TIME.toString());
+    if (tStr == null) {
+      return WorkflowContext.UNSTARTED;
+    }
+    return Long.parseLong(tStr);
+  }
+
   public void setPartitionFinishTime(int p, long t) {
     Map<String, String> map = getMapField(p);
     map.put(ContextProperties.FINISH_TIME.toString(), String.valueOf(t));
@@ -129,11 +146,11 @@ public class JobContext extends HelixProperty {
   public long getPartitionFinishTime(int p) {
     Map<String, String> map = getMapField(p);
     if (map == null) {
-      return -1;
+      return WorkflowContext.UNFINISHED;
     }
     String tStr = map.get(ContextProperties.FINISH_TIME.toString());
     if (tStr == null) {
-      return -1;
+      return WorkflowContext.UNFINISHED;
     }
     return Long.parseLong(tStr);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/ddefebb2/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index b02089f..0f34178 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -421,6 +421,7 @@ public class JobRebalancer extends TaskRebalancer {
             excludeSet.add(pId);
             jobCtx.setAssignedParticipant(pId, instance);
             jobCtx.setPartitionState(pId, TaskPartitionState.INIT);
+            jobCtx.setPartitionStartTime(pId, System.currentTimeMillis());
             LOG.debug(String.format("Setting task partition %s state to %s on instance %s.", pName,
                 TaskPartitionState.RUNNING, instance));
           }

http://git-wip-us.apache.org/repos/asf/helix/blob/ddefebb2/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java b/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
index 0e0a283..9c1f77a 100644
--- a/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
+++ b/helix-core/src/main/java/org/apache/helix/task/WorkflowContext.java
@@ -36,6 +36,7 @@ public class WorkflowContext extends HelixProperty {
   public static final String FINISH_TIME = "FINISH_TIME";
   public static final String JOB_STATES = "JOB_STATES";
   public static final String LAST_SCHEDULED_WORKFLOW = "LAST_SCHEDULED_WORKFLOW";
+  public static final int UNSTARTED = -1;
   public static final int UNFINISHED = -1;
 
   public WorkflowContext(ZNRecord record) {