You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by zz...@apache.org on 2013/12/07 00:25:43 UTC

[3/3] git commit: [HELIX-336] Add support for task framework, rb=16071

[HELIX-336] Add support for task framework, rb=16071


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

Branch: refs/heads/master
Commit: 80fc2be5edace93ad44d878da5f22bf046907806
Parents: 69de0f2
Author: zzhang <zz...@uci.edu>
Authored: Fri Dec 6 15:25:33 2013 -0800
Committer: zzhang <zz...@uci.edu>
Committed: Fri Dec 6 15:25:33 2013 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/helix/ZNRecord.java    |  13 +-
 .../helix/api/accessor/ResourceAccessor.java    |   2 +-
 .../stages/CurrentStateComputationStage.java    |   6 +
 .../stages/ExternalViewComputeStage.java        |   1 +
 .../stages/MessageSelectionStage.java           |   6 +
 .../controller/stages/ResourceCurrentState.java |  84 +++
 .../handling/HelixStateTransitionHandler.java   |  79 ++-
 .../messaging/handling/HelixTaskResult.java     |  11 +-
 .../org/apache/helix/model/CurrentState.java    |  52 ++
 .../participant/statemachine/StateModel.java    |  21 +
 .../java/org/apache/helix/task/TargetState.java |  39 ++
 .../main/java/org/apache/helix/task/Task.java   |  41 ++
 .../java/org/apache/helix/task/TaskConfig.java  | 292 ++++++++
 .../org/apache/helix/task/TaskConstants.java    |  42 ++
 .../java/org/apache/helix/task/TaskContext.java | 135 ++++
 .../java/org/apache/helix/task/TaskDag.java     | 152 +++++
 .../java/org/apache/helix/task/TaskDriver.java  | 361 ++++++++++
 .../java/org/apache/helix/task/TaskFactory.java |  32 +
 .../apache/helix/task/TaskPartitionState.java   |  42 ++
 .../org/apache/helix/task/TaskRebalancer.java   | 682 +++++++++++++++++++
 .../java/org/apache/helix/task/TaskResult.java  |  70 ++
 .../java/org/apache/helix/task/TaskRunner.java  | 174 +++++
 .../java/org/apache/helix/task/TaskState.java   |  42 ++
 .../org/apache/helix/task/TaskStateModel.java   | 240 +++++++
 .../helix/task/TaskStateModelFactory.java       |  42 ++
 .../java/org/apache/helix/task/TaskUtil.java    | 179 +++++
 .../java/org/apache/helix/task/Workflow.java    | 248 +++++++
 .../org/apache/helix/task/WorkflowConfig.java   | 113 +++
 .../org/apache/helix/task/WorkflowContext.java  | 125 ++++
 .../org/apache/helix/task/beans/TaskBean.java   |  40 ++
 .../apache/helix/task/beans/WorkflowBean.java   |  31 +
 .../org/apache/helix/tools/ClusterSetup.java    |   4 +-
 .../helix/tools/StateModelConfigGenerator.java  | 126 ++++
 .../java/org/apache/helix/TestZNRecord.java     |  22 +
 .../integration/ZkIntegrationTestBase.java      |   3 +-
 .../integration/task/TestTaskRebalancer.java    | 306 +++++++++
 .../task/TestTaskRebalancerStopResume.java      | 209 ++++++
 .../apache/helix/integration/task/TestUtil.java |  70 ++
 .../integration/task/WorkflowGenerator.java     |  85 +++
 39 files changed, 4206 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/ZNRecord.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/ZNRecord.java b/helix-core/src/main/java/org/apache/helix/ZNRecord.java
index 37cd5eb..e4fe00f 100644
--- a/helix-core/src/main/java/org/apache/helix/ZNRecord.java
+++ b/helix-core/src/main/java/org/apache/helix/ZNRecord.java
@@ -591,8 +591,19 @@ public class ZNRecord {
     }
 
     for (String key : value.getMapFields().keySet()) {
-      if (mapFields.containsKey(key)) {
+      Map<String, String> map = value.getMapField(key);
+      if (map == null) {
         mapFields.remove(key);
+      } else {
+        Map<String, String> nestedMap = mapFields.get(key);
+        if (nestedMap != null) {
+          for (String mapKey : map.keySet()) {
+            nestedMap.remove(mapKey);
+          }
+          if (nestedMap.size() == 0) {
+            mapFields.remove(key);
+          }
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
index b308b98..80c5b16 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
@@ -380,7 +380,7 @@ public class ResourceAccessor {
    * @param batchMessageMode true if batch messaging allowed, false otherwise
    * @return IdealState, or null
    */
-  static IdealState rebalancerConfigToIdealState(RebalancerConfig config, int bucketSize,
+  public static IdealState rebalancerConfigToIdealState(RebalancerConfig config, int bucketSize,
       boolean batchMessageMode) {
     PartitionedRebalancerConfig partitionedConfig = PartitionedRebalancerConfig.from(config);
     if (partitionedConfig != null) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
index c036b14..8235173 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
@@ -129,6 +129,12 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
         for (PartitionId partitionId : partitionStateMap.keySet()) {
           currentStateOutput.setCurrentState(resourceId, partitionId, participantId,
               curState.getState(partitionId));
+
+          currentStateOutput.setRequestedState(resourceId, partitionId, participantId,
+              curState.getRequestedState(partitionId));
+
+          currentStateOutput.setInfo(resourceId, partitionId, participantId,
+              curState.getInfo(partitionId));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
index 7704378..977b661 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
@@ -139,6 +139,7 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
         // scheduler
         // message, and then remove the partitions from the ideal state
         if (rebalancerConfig != null
+            && rebalancerConfig.getStateModelDefId() != null
             && rebalancerConfig.getStateModelDefId().equalsIgnoreCase(
                 StateModelDefId.SchedulerTaskQueue)) {
           updateScheduledTaskStatus(resourceId, view, manager, schedulerTaskConfig);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
index 9adc833..966160c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
@@ -111,6 +111,12 @@ public class MessageSelectionStage extends AbstractBaseStage {
       StateModelDefinition stateModelDef =
           stateModelDefMap.get(resource.getRebalancerConfig().getStateModelDefId());
 
+      if (stateModelDef == null) {
+        LOG.info("resource: " + resourceId
+            + " doesn't have state-model-def; e.g. we add a resource config but not add the resource in ideal-states");
+        continue;
+      }
+
       // TODO have a logical model for transition
       Map<String, Integer> stateTransitionPriorities = getStateTransitionPriorityMap(stateModelDef);
       Resource configResource = cluster.getResource(resourceId);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
index f04afd0..f4019ac 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
@@ -57,6 +57,19 @@ public class ResourceCurrentState {
   private final Map<ResourceId, CurrentState> _curStateMetaMap;
 
   /**
+   * Contains per-resource maps of partition -> (instance, requested_state). This corresponds to the
+   * REQUESTED_STATE field in the CURRENTSTATES node.
+   */
+  private final Map<ResourceId, Map<PartitionId, Map<ParticipantId, State>>> _requestedStateMap;
+
+  /**
+   * Contains per-resource maps of partition -> (instance, info). This corresponds to the INFO field
+   * in the CURRENTSTATES node. This is information returned by state transition methods on the
+   * participants. It may be used by the rebalancer.
+   */
+  private final Map<ResourceId, Map<PartitionId, Map<ParticipantId, String>>> _infoMap;
+
+  /**
    * construct
    */
   public ResourceCurrentState() {
@@ -65,6 +78,9 @@ public class ResourceCurrentState {
     _resourceStateModelMap = new HashMap<ResourceId, StateModelDefId>();
     _curStateMetaMap = new HashMap<ResourceId, CurrentState>();
 
+    _requestedStateMap = new HashMap<ResourceId, Map<PartitionId, Map<ParticipantId, State>>>();
+    _infoMap = new HashMap<ResourceId, Map<PartitionId, Map<ParticipantId, String>>>();
+
   }
 
   /**
@@ -270,4 +286,72 @@ public class ResourceCurrentState {
 
   }
 
+  /**
+   * @param resourceId
+   * @param partitionId
+   * @param participantId
+   * @param state
+   */
+  public void setRequestedState(ResourceId resourceId, PartitionId partitionId,
+      ParticipantId participantId, State state) {
+    if (!_requestedStateMap.containsKey(resourceId)) {
+      _requestedStateMap.put(resourceId, new HashMap<PartitionId, Map<ParticipantId, State>>());
+    }
+    if (!_requestedStateMap.get(resourceId).containsKey(partitionId)) {
+      _requestedStateMap.get(resourceId).put(partitionId, new HashMap<ParticipantId, State>());
+    }
+    _requestedStateMap.get(resourceId).get(partitionId).put(participantId, state);
+  }
+
+  /**
+   * @param resourceId
+   * @param partitionId
+   * @param participantId
+   * @param state
+   */
+  public void setInfo(ResourceId resourceId, PartitionId partitionId, ParticipantId participantId,
+      String info) {
+    if (!_infoMap.containsKey(resourceId)) {
+      _infoMap.put(resourceId, new HashMap<PartitionId, Map<ParticipantId, String>>());
+    }
+    if (!_infoMap.get(resourceId).containsKey(partitionId)) {
+      _infoMap.get(resourceId).put(partitionId, new HashMap<ParticipantId, String>());
+    }
+    _infoMap.get(resourceId).get(partitionId).put(participantId, info);
+  }
+
+  /**
+   * @param resourceId
+   * @param partitionId
+   * @param participantId
+   * @return
+   */
+  public State getRequestedState(ResourceId resourceId, PartitionId partitionId,
+      ParticipantId participantId) {
+    Map<PartitionId, Map<ParticipantId, State>> map = _requestedStateMap.get(resourceId);
+    if (map != null) {
+      Map<ParticipantId, State> instanceStateMap = map.get(partitionId);
+      if (instanceStateMap != null) {
+        return instanceStateMap.get(participantId);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * @param resourceId
+   * @param partitionId
+   * @param participantId
+   * @return
+   */
+  public String getInfo(ResourceId resourceId, PartitionId partitionId, ParticipantId participantId) {
+    Map<PartitionId, Map<ParticipantId, String>> map = _infoMap.get(resourceId);
+    if (map != null) {
+      Map<ParticipantId, String> instanceStateMap = map.get(partitionId);
+      if (instanceStateMap != null) {
+        return instanceStateMap.get(participantId);
+      }
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
index 8381f4a..55d8965 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
@@ -25,6 +25,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.HelixAdmin;
@@ -36,6 +38,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.ZNRecordBucketizer;
 import org.apache.helix.ZNRecordDelta;
 import org.apache.helix.ZNRecordDelta.MergeOperation;
@@ -60,7 +63,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
     }
   }
 
-  private static Logger logger = Logger.getLogger(HelixStateTransitionHandler.class);
+  private static final Logger logger = Logger.getLogger(HelixStateTransitionHandler.class);
   private final StateModel _stateModel;
   StatusUpdateUtil _statusUpdateUtil;
   private final StateModelParser _transitionMethodFinder;
@@ -113,6 +116,45 @@ public class HelixStateTransitionHandler extends MessageHandler {
       logger.error(errorMessage);
       throw new HelixStateMismatchException(errorMessage);
     }
+
+    /**
+     * Reset the REQUESTED_STATE property if it exists.
+     * ideally we should merge all current-state update into one zk-write
+     */
+    if (_stateModel.getRequestedState() != null) {
+      try {
+        String instance = _manager.getInstanceName();
+        String sessionId = _message.getTgtSessionId();
+        String resource = _message.getResourceName();
+        String partitionName = _message.getPartitionName();
+        ZNRecordBucketizer bucketizer = new ZNRecordBucketizer(_message.getBucketSize());
+        PropertyKey key =
+            accessor.keyBuilder().currentState(instance, sessionId, resource,
+                bucketizer.getBucketName(partitionName));
+        ZNRecord rec = new ZNRecord(resource);
+        Map<String, String> map = new TreeMap<String, String>();
+        map.put(CurrentState.CurrentStateProperty.REQUESTED_STATE.name(), null);
+        rec.getMapFields().put(partitionName, map);
+        ZNRecordDelta delta = new ZNRecordDelta(rec, ZNRecordDelta.MergeOperation.SUBTRACT);
+        List<ZNRecordDelta> deltaList = new ArrayList<ZNRecordDelta>();
+        deltaList.add(delta);
+        CurrentState currStateUpdate = new CurrentState(resource);
+        currStateUpdate.setDeltaList(deltaList);
+        // Update the ZK current state of the node
+        accessor.updateProperty(key, currStateUpdate);
+        _stateModel.setRequestedState(null);
+      } catch (Exception e) {
+        logger.error(
+            "Error when removing " + CurrentState.CurrentStateProperty.REQUESTED_STATE.name()
+                + " from current state.", e);
+        StateTransitionError error =
+            new StateTransitionError(ErrorType.FRAMEWORK, ErrorCode.ERROR, e);
+        _stateModel.rollbackOnError(_message, _notificationContext, error);
+        _statusUpdateUtil.logError(_message, HelixStateTransitionHandler.class, e,
+            "Error when removing " + CurrentState.CurrentStateProperty.REQUESTED_STATE.name()
+                + " from current state.", accessor);
+      }
+    }
   }
 
   void postHandleMessage() {
@@ -141,6 +183,9 @@ public class HelixStateTransitionHandler extends MessageHandler {
       return;
     }
 
+    // Set the INFO property.
+    _currentStateDelta.setInfo(partitionId, taskResult.getInfo());
+
     if (taskResult.isSuccess()) {
       // String fromState = message.getFromState();
       State toState = _message.getTypedToState();
@@ -150,10 +195,11 @@ public class HelixStateTransitionHandler extends MessageHandler {
         // for "OnOfflineToDROPPED" message, we need to remove the resource key record
         // from the current state of the instance because the resource key is dropped.
         // In the state model it will be stayed as "OFFLINE", which is OK.
-        ZNRecordDelta delta =
-            new ZNRecordDelta(_currentStateDelta.getRecord(), MergeOperation.SUBTRACT);
-        // Don't subtract simple fields since they contain stateModelDefRef
-        delta._record.getSimpleFields().clear();
+
+        ZNRecord rec = new ZNRecord(_currentStateDelta.getId());
+        // remove mapField keyed by partitionId
+        rec.setMapField(partitionId.stringify(), null);
+        ZNRecordDelta delta = new ZNRecordDelta(rec, MergeOperation.SUBTRACT);
 
         List<ZNRecordDelta> deltaList = new ArrayList<ZNRecordDelta>();
         deltaList.add(delta);
@@ -190,7 +236,8 @@ public class HelixStateTransitionHandler extends MessageHandler {
         _stateModel.updateState(HelixDefinedState.ERROR.toString());
 
         // if we have errors transit from ERROR state, disable the partition
-        if (_message.getTypedFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
+        if (_message.getTypedFromState().toString()
+            .equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
           disablePartition();
         }
       }
@@ -296,10 +343,21 @@ public class HelixStateTransitionHandler extends MessageHandler {
                 Message.class, NotificationContext.class
             });
     if (methodToInvoke != null) {
-      methodToInvoke.invoke(_stateModel, new Object[] {
+      logger.info(String.format(
+          "Instance %s, partition %s received state transition from %s to %s on session %s.",
+          message.getTgtName(), message.getPartitionName(), message.getFromState(),
+          message.getToState(), message.getTgtSessionId()));
+      Object result = methodToInvoke.invoke(_stateModel, new Object[] {
           message, context
       });
       taskResult.setSuccess(true);
+      String resultStr;
+      if (result == null || result instanceof Void) {
+        resultStr = "";
+      } else {
+        resultStr = result.toString();
+      }
+      taskResult.setInfo(resultStr);
     } else {
       String errorMessage =
           "Unable to find method for transition from " + fromState + " to " + toState + " in "
@@ -335,11 +393,12 @@ public class HelixStateTransitionHandler extends MessageHandler {
         _stateModel.updateState(HelixDefinedState.ERROR.toString());
 
         // if transit from ERROR state, disable the partition
-        if (_message.getTypedFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
+        if (_message.getTypedFromState().toString()
+            .equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
           disablePartition();
         }
-        accessor.updateProperty(keyBuilder.currentState(instanceName, _message.getTypedTgtSessionId()
-            .stringify(), resourceId.stringify()), currentStateDelta);
+        accessor.updateProperty(keyBuilder.currentState(instanceName, _message
+            .getTypedTgtSessionId().stringify(), resourceId.stringify()), currentStateDelta);
       }
     } finally {
       StateTransitionError error = new StateTransitionError(type, code, e);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
index 22c4fcd..df8a53e 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
@@ -26,7 +26,8 @@ public class HelixTaskResult {
 
   private boolean _success;
   private String _message = "";
-  private Map<String, String> _taskResultMap = new HashMap<String, String>();
+  private String _info = "";
+  private final Map<String, String> _taskResultMap = new HashMap<String, String>();
   private boolean _interrupted = false;
   Exception _exception = null;
 
@@ -65,4 +66,12 @@ public class HelixTaskResult {
   public Exception getException() {
     return _exception;
   }
+
+  public String getInfo() {
+    return _info;
+  }
+
+  public void setInfo(String info) {
+    _info = info;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
index 5c9bcbc..05d81c3 100644
--- a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
+++ b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
@@ -45,6 +45,8 @@ public class CurrentState extends HelixProperty {
   public enum CurrentStateProperty {
     SESSION_ID,
     CURRENT_STATE,
+    REQUESTED_STATE,
+    INFO,
     STATE_MODEL_DEF,
     STATE_MODEL_FACTORY_NAME,
     RESOURCE // ,
@@ -328,4 +330,54 @@ public class CurrentState extends HelixProperty {
     }
     return rawMap;
   }
+
+  /**
+   * @param partitionId
+   * @return
+   */
+  public String getInfo(PartitionId partitionId) {
+    Map<String, String> mapField = _record.getMapField(partitionId.stringify());
+    if (mapField != null) {
+      return mapField.get(CurrentStateProperty.INFO.name());
+    }
+    return null;
+  }
+
+  /**
+   * @param partitionId
+   * @return
+   */
+  public State getRequestedState(PartitionId partitionId) {
+    Map<String, String> mapField = _record.getMapField(partitionId.stringify());
+    if (mapField != null) {
+      return State.from(mapField.get(CurrentStateProperty.REQUESTED_STATE.name()));
+    }
+    return null;
+  }
+
+  /**
+   * @param partitionId
+   * @param info
+   */
+  public void setInfo(PartitionId partitionId, String info) {
+    Map<String, Map<String, String>> mapFields = _record.getMapFields();
+    String partitionName = partitionId.stringify();
+    if (mapFields.get(partitionName) == null) {
+      mapFields.put(partitionName, new TreeMap<String, String>());
+    }
+    mapFields.get(partitionName).put(CurrentStateProperty.INFO.name(), info);
+  }
+
+  /**
+   * @param partitionId
+   * @param state
+   */
+  public void setRequestedState(PartitionId partitionId, State state) {
+    Map<String, Map<String, String>> mapFields = _record.getMapFields();
+    String partitionName = partitionId.stringify();
+    if (mapFields.get(partitionName) == null) {
+      mapFields.put(partitionName, new TreeMap<String, String>());
+    }
+    mapFields.get(partitionName).put(CurrentStateProperty.REQUESTED_STATE.name(), state.toString());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModel.java b/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModel.java
index b88262b..9717340 100644
--- a/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModel.java
+++ b/helix-core/src/main/java/org/apache/helix/participant/statemachine/StateModel.java
@@ -31,6 +31,11 @@ public abstract class StateModel {
   // StateModel with initial state other than OFFLINE should override this field
   protected String _currentState = DEFAULT_INITIAL_STATE;
 
+  /**
+   * requested-state is used (e.g. by task-framework) to request next state
+   */
+  protected String _requestedState = null;
+
   public String getCurrentState() {
     return _currentState;
   }
@@ -47,6 +52,22 @@ public abstract class StateModel {
   }
 
   /**
+   * Get requested-state
+   * @return requested-state
+   */
+  public String getRequestedState() {
+    return _requestedState;
+  }
+
+  /**
+   * Set requested-state
+   * @param requestedState
+   */
+  public void setRequestedState(String requestedState) {
+    _requestedState = requestedState;
+  }
+
+  /**
    * Called when error occurs in state transition
    * TODO:enforce subclass to write this
    * @param message

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/task/TargetState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TargetState.java b/helix-core/src/main/java/org/apache/helix/task/TargetState.java
new file mode 100644
index 0000000..0551d6c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TargetState.java
@@ -0,0 +1,39 @@
+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.
+ */
+
+/**
+ * Enumeration of target states for a task.
+ */
+public enum TargetState {
+  /**
+   * Indicates that the rebalancer must start/resume the task.
+   */
+  START,
+  /**
+   * Indicates that the rebalancer should stop any running task partitions and cease doing any
+   * further task assignments.
+   */
+  STOP,
+  /**
+   * Indicates that the rebalancer must delete this task.
+   */
+  DELETE
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/task/Task.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/Task.java b/helix-core/src/main/java/org/apache/helix/task/Task.java
new file mode 100644
index 0000000..207fd96
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/Task.java
@@ -0,0 +1,41 @@
+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.
+ */
+
+/**
+ * The interface that is to be implemented by a specific task implementation.
+ */
+public interface Task {
+  /**
+   * Execute the task.
+   * @return A {@link TaskResult} object indicating the status of the task and any additional
+   *         context
+   *         information that
+   *         can be interpreted by the specific {@link Task} implementation.
+   */
+  TaskResult run();
+
+  /**
+   * Signals the task to stop execution. The task implementation should carry out any clean up
+   * actions that may be
+   * required and return from the {@link #run()} method.
+   */
+  void cancel();
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/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
new file mode 100644
index 0000000..8d1c4bb
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java
@@ -0,0 +1,292 @@
+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 com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.task.Workflow.WorkflowEnum;
+
+/**
+ * Provides a typed interface to task configurations.
+ */
+public class TaskConfig {
+  // // Property names ////
+
+  /** The name of the workflow to which the task belongs. */
+  public static final String WORKFLOW_ID = "WorkflowID";
+  /** The name of the target resource. */
+  public static final String TARGET_RESOURCE = "TargetResource";
+  /**
+   * The set of the target partition states. The value must be a comma-separated list of partition
+   * states.
+   */
+  public static final String TARGET_PARTITION_STATES = "TargetPartitionStates";
+  /**
+   * The set of the target partition ids. The value must be a comma-separated list of partition ids.
+   */
+  public static final String TARGET_PARTITIONS = "TargetPartitions";
+  /** The command that is to be run by participants. */
+  public static final String COMMAND = "Command";
+  /** The command configuration to be used by the task partitions. */
+  public static final String COMMAND_CONFIG = "CommandConfig";
+  /** The timeout for a task partition. */
+  public static final String TIMEOUT_PER_PARTITION = "TimeoutPerPartition";
+  /** The maximum number of times the task rebalancer may attempt to execute a task partitions. */
+  public static final String MAX_ATTEMPTS_PER_PARTITION = "MaxAttemptsPerPartition";
+  /** The number of concurrent tasks that are allowed to run on an instance. */
+  public static final String NUM_CONCURRENT_TASKS_PER_INSTANCE = "ConcurrentTasksPerInstance";
+
+  // // Default property values ////
+
+  public static final long DEFAULT_TIMEOUT_PER_PARTITION = 60 * 60 * 1000; // 1 hr.
+  public static final int DEFAULT_MAX_ATTEMPTS_PER_PARTITION = 10;
+  public static final int DEFAULT_NUM_CONCURRENT_TASKS_PER_INSTANCE = 1;
+
+  private final String _workflow;
+  private final String _targetResource;
+  private final List<Integer> _targetPartitions;
+  private final Set<String> _targetPartitionStates;
+  private final String _command;
+  private final String _commandConfig;
+  private final long _timeoutPerPartition;
+  private final int _numConcurrentTasksPerInstance;
+  private final int _maxAttemptsPerPartition;
+
+  private TaskConfig(String workflow, String targetResource, List<Integer> targetPartitions,
+      Set<String> targetPartitionStates, String command, String commandConfig,
+      long timeoutPerPartition, int numConcurrentTasksPerInstance, int maxAttemptsPerPartition) {
+    _workflow = workflow;
+    _targetResource = targetResource;
+    _targetPartitions = targetPartitions;
+    _targetPartitionStates = targetPartitionStates;
+    _command = command;
+    _commandConfig = commandConfig;
+    _timeoutPerPartition = timeoutPerPartition;
+    _numConcurrentTasksPerInstance = numConcurrentTasksPerInstance;
+    _maxAttemptsPerPartition = maxAttemptsPerPartition;
+  }
+
+  public String getWorkflow() {
+    return _workflow == null ? WorkflowEnum.UNSPECIFIED.name() : _workflow;
+  }
+
+  public String getTargetResource() {
+    return _targetResource;
+  }
+
+  public List<Integer> getTargetPartitions() {
+    return _targetPartitions;
+  }
+
+  public Set<String> getTargetPartitionStates() {
+    return _targetPartitionStates;
+  }
+
+  public String getCommand() {
+    return _command;
+  }
+
+  public String getCommandConfig() {
+    return _commandConfig;
+  }
+
+  public long getTimeoutPerPartition() {
+    return _timeoutPerPartition;
+  }
+
+  public int getNumConcurrentTasksPerInstance() {
+    return _numConcurrentTasksPerInstance;
+  }
+
+  public int getMaxAttemptsPerPartition() {
+    return _maxAttemptsPerPartition;
+  }
+
+  public Map<String, String> getResourceConfigMap() {
+    Map<String, String> cfgMap = new HashMap<String, String>();
+    cfgMap.put(TaskConfig.WORKFLOW_ID, _workflow);
+    cfgMap.put(TaskConfig.COMMAND, _command);
+    cfgMap.put(TaskConfig.COMMAND_CONFIG, _commandConfig);
+    cfgMap.put(TaskConfig.TARGET_RESOURCE, _targetResource);
+    cfgMap.put(TaskConfig.TARGET_PARTITION_STATES, Joiner.on(",").join(_targetPartitionStates));
+    if (_targetPartitions != null) {
+      cfgMap.put(TaskConfig.TARGET_PARTITIONS, Joiner.on(",").join(_targetPartitions));
+    }
+    cfgMap.put(TaskConfig.TIMEOUT_PER_PARTITION, "" + _timeoutPerPartition);
+    cfgMap.put(TaskConfig.MAX_ATTEMPTS_PER_PARTITION, "" + _maxAttemptsPerPartition);
+
+    return cfgMap;
+  }
+
+  /**
+   * A builder for {@link TaskConfig}. Validates the configurations.
+   */
+  public static class Builder {
+    private String _workflow;
+    private String _targetResource;
+    private List<Integer> _targetPartitions;
+    private Set<String> _targetPartitionStates;
+    private String _command;
+    private String _commandConfig;
+    private long _timeoutPerPartition = DEFAULT_TIMEOUT_PER_PARTITION;
+    private int _numConcurrentTasksPerInstance = DEFAULT_NUM_CONCURRENT_TASKS_PER_INSTANCE;
+    private int _maxAttemptsPerPartition = DEFAULT_MAX_ATTEMPTS_PER_PARTITION;
+
+    public TaskConfig build() {
+      validate();
+
+      return new TaskConfig(_workflow, _targetResource, _targetPartitions, _targetPartitionStates,
+          _command, _commandConfig, _timeoutPerPartition, _numConcurrentTasksPerInstance,
+          _maxAttemptsPerPartition);
+    }
+
+    /**
+     * Convenience method to build a {@link TaskConfig} from a {@code Map&lt;String, String&gt;}.
+     * @param cfg A map of property names to their string representations.
+     * @return A {@link Builder}.
+     */
+    public static Builder fromMap(Map<String, String> cfg) {
+      Builder b = new Builder();
+      if (cfg.containsKey(WORKFLOW_ID)) {
+        b.setWorkflow(cfg.get(WORKFLOW_ID));
+      }
+      if (cfg.containsKey(TARGET_RESOURCE)) {
+        b.setTargetResource(cfg.get(TARGET_RESOURCE));
+      }
+      if (cfg.containsKey(TARGET_PARTITIONS)) {
+        b.setTargetPartitions(csvToIntList(cfg.get(TARGET_PARTITIONS)));
+      }
+      if (cfg.containsKey(TARGET_PARTITION_STATES)) {
+        b.setTargetPartitionStates(new HashSet<String>(Arrays.asList(cfg.get(
+            TARGET_PARTITION_STATES).split(","))));
+      }
+      if (cfg.containsKey(COMMAND)) {
+        b.setCommand(cfg.get(COMMAND));
+      }
+      if (cfg.containsKey(COMMAND_CONFIG)) {
+        b.setCommandConfig(cfg.get(COMMAND_CONFIG));
+      }
+      if (cfg.containsKey(TIMEOUT_PER_PARTITION)) {
+        b.setTimeoutPerPartition(Long.parseLong(cfg.get(TIMEOUT_PER_PARTITION)));
+      }
+      if (cfg.containsKey(NUM_CONCURRENT_TASKS_PER_INSTANCE)) {
+        b.setNumConcurrentTasksPerInstance(Integer.parseInt(cfg
+            .get(NUM_CONCURRENT_TASKS_PER_INSTANCE)));
+      }
+      if (cfg.containsKey(MAX_ATTEMPTS_PER_PARTITION)) {
+        b.setMaxAttemptsPerPartition(Integer.parseInt(cfg.get(MAX_ATTEMPTS_PER_PARTITION)));
+      }
+
+      return b;
+    }
+
+    public Builder setWorkflow(String v) {
+      _workflow = v;
+      return this;
+    }
+
+    public Builder setTargetResource(String v) {
+      _targetResource = v;
+      return this;
+    }
+
+    public Builder setTargetPartitions(List<Integer> v) {
+      _targetPartitions = ImmutableList.copyOf(v);
+      return this;
+    }
+
+    public Builder setTargetPartitionStates(Set<String> v) {
+      _targetPartitionStates = ImmutableSet.copyOf(v);
+      return this;
+    }
+
+    public Builder setCommand(String v) {
+      _command = v;
+      return this;
+    }
+
+    public Builder setCommandConfig(String v) {
+      _commandConfig = v;
+      return this;
+    }
+
+    public Builder setTimeoutPerPartition(long v) {
+      _timeoutPerPartition = v;
+      return this;
+    }
+
+    public Builder setNumConcurrentTasksPerInstance(int v) {
+      _numConcurrentTasksPerInstance = v;
+      return this;
+    }
+
+    public Builder setMaxAttemptsPerPartition(int v) {
+      _maxAttemptsPerPartition = v;
+      return this;
+    }
+
+    private void validate() {
+      if (_targetResource == null) {
+        throw new IllegalArgumentException(String.format("%s cannot be null", TARGET_RESOURCE));
+      }
+      if (_targetPartitionStates != null && _targetPartitionStates.isEmpty()) {
+        throw new IllegalArgumentException(String.format("%s cannot be an empty set",
+            TARGET_PARTITION_STATES));
+      }
+      if (_command == null) {
+        throw new IllegalArgumentException(String.format("%s cannot be null", COMMAND));
+      }
+      if (_timeoutPerPartition < 0) {
+        throw new IllegalArgumentException(String.format("%s has invalid value %s",
+            TIMEOUT_PER_PARTITION, _timeoutPerPartition));
+      }
+      if (_numConcurrentTasksPerInstance < 1) {
+        throw new IllegalArgumentException(String.format("%s has invalid value %s",
+            NUM_CONCURRENT_TASKS_PER_INSTANCE, _numConcurrentTasksPerInstance));
+      }
+      if (_maxAttemptsPerPartition < 1) {
+        throw new IllegalArgumentException(String.format("%s has invalid value %s",
+            MAX_ATTEMPTS_PER_PARTITION, _maxAttemptsPerPartition));
+      }
+      if (_workflow == null) {
+        throw new IllegalArgumentException(String.format("%s cannot be null", WORKFLOW_ID));
+      }
+    }
+
+    private static List<Integer> csvToIntList(String csv) {
+      String[] vals = csv.split(",");
+      List<Integer> l = new ArrayList<Integer>();
+      for (String v : vals) {
+        l.add(Integer.parseInt(v));
+      }
+
+      return l;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/task/TaskConstants.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskConstants.java b/helix-core/src/main/java/org/apache/helix/task/TaskConstants.java
new file mode 100644
index 0000000..305323d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskConstants.java
@@ -0,0 +1,42 @@
+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.
+ */
+
+/**
+ * Constants used in the task framework.
+ */
+public class TaskConstants {
+  /**
+   * The name of the {@link Task} state model.
+   */
+  public static final String STATE_MODEL_NAME = "Task";
+  /**
+   * Field in workflow resource config housing dag
+   */
+  public static final String WORKFLOW_DAG_FIELD = "dag";
+  /**
+   * Field in workflow resource config for flow name
+   */
+  public static final String WORKFLOW_NAME_FIELD = "name";
+  /**
+   * The root property store path at which the {@link TaskRebalancer} stores context information.
+   */
+  public static final String REBALANCER_CONTEXT_ROOT = "/TaskRebalancer";
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/task/TaskContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskContext.java b/helix-core/src/main/java/org/apache/helix/task/TaskContext.java
new file mode 100644
index 0000000..d416a86
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskContext.java
@@ -0,0 +1,135 @@
+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.Map;
+import java.util.TreeMap;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.ZNRecord;
+
+/**
+ * Provides a typed interface to the context information stored by {@link TaskRebalancer} in the
+ * Helix property store.
+ */
+public class TaskContext extends HelixProperty {
+  public static final String START_TIME = "START_TIME";
+  public static final String PARTITION_STATE = "STATE";
+  public static final String NUM_ATTEMPTS = "NUM_ATTEMPTS";
+  public static final String FINISH_TIME = "FINISH_TIME";
+
+  public TaskContext(ZNRecord record) {
+    super(record);
+  }
+
+  public void setStartTime(long t) {
+    _record.setSimpleField(START_TIME, String.valueOf(t));
+  }
+
+  public long getStartTime() {
+    String tStr = _record.getSimpleField(START_TIME);
+    if (tStr == null) {
+      return -1;
+    }
+
+    return Long.parseLong(tStr);
+  }
+
+  public void setPartitionState(int p, TaskPartitionState s) {
+    String pStr = String.valueOf(p);
+    Map<String, String> map = _record.getMapField(pStr);
+    if (map == null) {
+      map = new TreeMap<String, String>();
+      _record.setMapField(pStr, map);
+    }
+    map.put(PARTITION_STATE, s.name());
+  }
+
+  public TaskPartitionState getPartitionState(int p) {
+    Map<String, String> map = _record.getMapField(String.valueOf(p));
+    if (map == null) {
+      return null;
+    }
+
+    String str = map.get(PARTITION_STATE);
+    if (str != null) {
+      return TaskPartitionState.valueOf(str);
+    } else {
+      return null;
+    }
+  }
+
+  public void setPartitionNumAttempts(int p, int n) {
+    String pStr = String.valueOf(p);
+    Map<String, String> map = _record.getMapField(pStr);
+    if (map == null) {
+      map = new TreeMap<String, String>();
+      _record.setMapField(pStr, map);
+    }
+    map.put(NUM_ATTEMPTS, String.valueOf(n));
+  }
+
+  public int incrementNumAttempts(int pId) {
+    int n = this.getPartitionNumAttempts(pId);
+    if (n < 0) {
+      n = 0;
+    }
+    n += 1;
+    this.setPartitionNumAttempts(pId, n);
+    return n;
+  }
+
+  public int getPartitionNumAttempts(int p) {
+    Map<String, String> map = _record.getMapField(String.valueOf(p));
+    if (map == null) {
+      return -1;
+    }
+
+    String nStr = map.get(NUM_ATTEMPTS);
+    if (nStr == null) {
+      return -1;
+    }
+
+    return Integer.parseInt(nStr);
+  }
+
+  public void setPartitionFinishTime(int p, long t) {
+    String pStr = String.valueOf(p);
+    Map<String, String> map = _record.getMapField(pStr);
+    if (map == null) {
+      map = new TreeMap<String, String>();
+      _record.setMapField(pStr, map);
+    }
+    map.put(FINISH_TIME, String.valueOf(t));
+  }
+
+  public long getPartitionFinishTime(int p) {
+    Map<String, String> map = _record.getMapField(String.valueOf(p));
+    if (map == null) {
+      return -1;
+    }
+
+    String tStr = map.get(FINISH_TIME);
+    if (tStr == null) {
+      return -1;
+    }
+
+    return Long.parseLong(tStr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/task/TaskDag.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDag.java b/helix-core/src/main/java/org/apache/helix/task/TaskDag.java
new file mode 100644
index 0000000..ab5bc62
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDag.java
@@ -0,0 +1,152 @@
+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 org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+/**
+ * Provides a convenient way to construct, traverse,
+ * and validate a task dependency graph
+ */
+public class TaskDag {
+  @JsonProperty("parentsToChildren")
+  private final Map<String, Set<String>> _parentsToChildren;
+
+  @JsonProperty("childrenToParents")
+  private final Map<String, Set<String>> _childrenToParents;
+
+  @JsonProperty("allNodes")
+  private final Set<String> _allNodes;
+
+  public static final TaskDag EMPTY_DAG = new TaskDag();
+
+  public TaskDag() {
+    _parentsToChildren = new TreeMap<String, Set<String>>();
+    _childrenToParents = new TreeMap<String, Set<String>>();
+    _allNodes = new TreeSet<String>();
+  }
+
+  public void addParentToChild(String parent, String child) {
+    if (!_parentsToChildren.containsKey(parent)) {
+      _parentsToChildren.put(parent, new TreeSet<String>());
+    }
+    _parentsToChildren.get(parent).add(child);
+
+    if (!_childrenToParents.containsKey(child)) {
+      _childrenToParents.put(child, new TreeSet<String>());
+    }
+    _childrenToParents.get(child).add(parent);
+
+    _allNodes.add(parent);
+    _allNodes.add(child);
+  }
+
+  public void addNode(String node) {
+    _allNodes.add(node);
+  }
+
+  public Map<String, Set<String>> getParentsToChildren() {
+    return _parentsToChildren;
+  }
+
+  public Map<String, Set<String>> getChildrenToParents() {
+    return _childrenToParents;
+  }
+
+  public Set<String> getAllNodes() {
+    return _allNodes;
+  }
+
+  public Set<String> getDirectChildren(String node) {
+    if (!_parentsToChildren.containsKey(node)) {
+      return Collections.emptySet();
+    }
+    return _parentsToChildren.get(node);
+  }
+
+  public Set<String> getDirectParents(String node) {
+    if (!_childrenToParents.containsKey(node)) {
+      return Collections.emptySet();
+    }
+    return _childrenToParents.get(node);
+  }
+
+  public String toJson() throws Exception {
+    return new ObjectMapper().writeValueAsString(this);
+  }
+
+  public static TaskDag fromJson(String json) {
+    try {
+      return new ObjectMapper().readValue(json, TaskDag.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Unable to parse json " + json + " into task dag");
+    }
+  }
+
+  /**
+   * Checks that dag contains no cycles and all nodes are reachable.
+   */
+  public void validate() {
+    Set<String> prevIteration = new TreeSet<String>();
+
+    // get all unparented nodes
+    for (String node : _allNodes) {
+      if (getDirectParents(node).isEmpty()) {
+        prevIteration.add(node);
+      }
+    }
+
+    // visit children nodes up to max iteration count, by which point we should have exited
+    // naturally
+    Set<String> allNodesReached = new TreeSet<String>();
+    int iterationCount = 0;
+    int maxIterations = _allNodes.size() + 1;
+
+    while (!prevIteration.isEmpty() && iterationCount < maxIterations) {
+      // construct set of all children reachable from prev iteration
+      Set<String> thisIteration = new TreeSet<String>();
+      for (String node : prevIteration) {
+        thisIteration.addAll(getDirectChildren(node));
+      }
+
+      allNodesReached.addAll(prevIteration);
+      prevIteration = thisIteration;
+      iterationCount++;
+    }
+
+    allNodesReached.addAll(prevIteration);
+
+    if (iterationCount >= maxIterations) {
+      throw new IllegalArgumentException("DAG invalid: cycles detected");
+    }
+
+    if (!allNodesReached.containsAll(_allNodes)) {
+      throw new IllegalArgumentException("DAG invalid: unreachable nodes found. Reachable set is "
+          + allNodesReached);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/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
new file mode 100644
index 0000000..d017134
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -0,0 +1,361 @@
+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.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.InstanceType;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.builder.CustomModeISBuilder;
+import org.apache.log4j.Logger;
+
+/**
+ * CLI for scheduling/canceling workflows
+ */
+public class TaskDriver {
+  /** For logging */
+  private static final Logger LOG = Logger.getLogger(TaskDriver.class);
+
+  /** Required option name for Helix endpoint */
+  private static final String ZK_ADDRESS = "zk";
+
+  /** Required option name for cluster against which to run task */
+  private static final String CLUSTER_NAME_OPTION = "cluster";
+
+  /** Required option name for task resource within target cluster */
+  private static final String RESOURCE_OPTION = "resource";
+
+  /** Field for specifying a workflow file when starting a job */
+  private static final String WORKFLOW_FILE_OPTION = "file";
+
+  private final HelixManager _manager;
+  private final HelixAdmin _admin;
+  private final String _clusterName;
+
+  /** Commands which may be parsed from the first argument to main */
+  private enum DriverCommand {
+    start,
+    stop,
+    delete,
+    resume,
+    list
+  }
+
+  public TaskDriver(HelixManager manager) {
+    _manager = manager;
+    _clusterName = manager.getClusterName();
+    _admin = manager.getClusterManagmentTool();
+  }
+
+  /**
+   * Parses the first argument as a driver command and the rest of the
+   * arguments are parsed based on that command. Constructs a Helix
+   * message and posts it to the controller
+   */
+  public static void main(String[] args) throws Exception {
+    String[] cmdArgs = Arrays.copyOfRange(args, 1, args.length);
+    CommandLine cl = parseOptions(cmdArgs, constructOptions(), args[0]);
+    String zkAddr = cl.getOptionValue(ZK_ADDRESS);
+    String clusterName = cl.getOptionValue(CLUSTER_NAME_OPTION);
+    String resource = cl.getOptionValue(RESOURCE_OPTION);
+
+    if (zkAddr == null || clusterName == null || resource == null) {
+      printUsage(constructOptions(), "[cmd]");
+      throw new IllegalArgumentException(
+          "zk, cluster, and resource must all be non-null for all commands");
+    }
+
+    HelixManager helixMgr =
+        HelixManagerFactory.getZKHelixManager(clusterName, "Admin", InstanceType.ADMINISTRATOR,
+            zkAddr);
+    helixMgr.connect();
+    TaskDriver driver = new TaskDriver(helixMgr);
+    try {
+      DriverCommand cmd = DriverCommand.valueOf(args[0]);
+      switch (cmd) {
+      case start:
+        if (cl.hasOption(WORKFLOW_FILE_OPTION)) {
+          driver.start(Workflow.parse(new File(cl.getOptionValue(WORKFLOW_FILE_OPTION))));
+        } else {
+          throw new IllegalArgumentException("Workflow file is required to start flow.");
+        }
+        break;
+      case stop:
+        driver.setTaskTargetState(resource, TargetState.STOP);
+        break;
+      case resume:
+        driver.setTaskTargetState(resource, TargetState.START);
+        break;
+      case delete:
+        driver.setTaskTargetState(resource, TargetState.DELETE);
+        break;
+      case list:
+        driver.list(resource);
+      default:
+        throw new IllegalArgumentException("Unknown command " + args[0]);
+      }
+    } catch (IllegalArgumentException e) {
+      LOG.error("Unknown driver command " + args[0]);
+      throw e;
+    }
+
+    helixMgr.disconnect();
+  }
+
+  /** Schedules a new workflow */
+  public void start(Workflow flow) throws Exception {
+    // TODO: check that namespace for workflow is available
+    LOG.info("Starting workflow " + flow.getName());
+    flow.validate();
+
+    String flowName = flow.getName();
+
+    // first, add workflow config to ZK
+    _admin.setConfig(TaskUtil.getResourceConfigScope(_clusterName, flowName),
+        flow.getResourceConfigMap());
+
+    // then schedule tasks
+    for (String task : flow.getTaskConfigs().keySet()) {
+      scheduleTask(task, TaskConfig.Builder.fromMap(flow.getTaskConfigs().get(task)).build());
+    }
+  }
+
+  /** Posts new task to cluster */
+  private void scheduleTask(String taskResource, TaskConfig taskConfig) throws Exception {
+    // Set up task resource based on partitions from target resource
+    int numPartitions =
+        _admin.getResourceIdealState(_clusterName, taskConfig.getTargetResource())
+            .getPartitionSet().size();
+    _admin.addResource(_clusterName, taskResource, numPartitions, TaskConstants.STATE_MODEL_NAME);
+    _admin.setConfig(TaskUtil.getResourceConfigScope(_clusterName, taskResource),
+        taskConfig.getResourceConfigMap());
+
+    // Push out new ideal state based on number of target partitions
+    CustomModeISBuilder builder = new CustomModeISBuilder(taskResource);
+    builder.setRebalancerMode(IdealState.RebalanceMode.USER_DEFINED);
+    builder.setNumReplica(1);
+    builder.setNumPartitions(numPartitions);
+    builder.setStateModel(TaskConstants.STATE_MODEL_NAME);
+    for (int i = 0; i < numPartitions; i++) {
+      builder.add(taskResource + "_" + i);
+    }
+    IdealState is = builder.build();
+    is.setRebalancerClassName(TaskRebalancer.class.getName());
+    _admin.setResourceIdealState(_clusterName, taskResource, is);
+  }
+
+  /** Public method to resume a task/workflow */
+  public void resume(String resource) {
+    setTaskTargetState(resource, TargetState.START);
+  }
+
+  /** Public method to stop a task/workflow */
+  public void stop(String resource) {
+    setTaskTargetState(resource, TargetState.STOP);
+  }
+
+  /** Public method to delete a task/workflow */
+  public void delete(String resource) {
+    setTaskTargetState(resource, TargetState.DELETE);
+  }
+
+  /** Helper function to change target state for a given task */
+  private void setTaskTargetState(String taskResource, TargetState state) {
+    HelixDataAccessor accessor = _manager.getHelixDataAccessor();
+    HelixProperty p = new HelixProperty(taskResource);
+    p.getRecord().setSimpleField(WorkflowConfig.TARGET_STATE, state.name());
+    accessor.updateProperty(accessor.keyBuilder().resourceConfig(taskResource), p);
+
+    invokeRebalance();
+  }
+
+  public void list(String resource) {
+    WorkflowConfig wCfg = TaskUtil.getWorkflowCfg(_manager, resource);
+    WorkflowContext wCtx = TaskUtil.getWorkflowContext(_manager, resource);
+
+    LOG.info("Workflow " + resource + " consists of the following tasks: "
+        + wCfg.getTaskDag().getAllNodes());
+    LOG.info("Current state of workflow is " + wCtx.getWorkflowState().name());
+    LOG.info("Task states are: ");
+    LOG.info("-------");
+    for (String task : wCfg.getTaskDag().getAllNodes()) {
+      LOG.info("Task " + task + " is " + wCtx.getTaskState(task));
+
+      // fetch task information
+      TaskContext tCtx = TaskUtil.getTaskContext(_manager, task);
+      TaskConfig tCfg = TaskUtil.getTaskCfg(_manager, task);
+
+      // calculate taskPartitions
+      List<Integer> partitions;
+      if (tCfg.getTargetPartitions() != null) {
+        partitions = tCfg.getTargetPartitions();
+      } else {
+        partitions = new ArrayList<Integer>();
+        for (String pStr : _admin.getResourceIdealState(_clusterName, tCfg.getTargetResource())
+            .getPartitionSet()) {
+          partitions
+              .add(Integer.parseInt(pStr.substring(pStr.lastIndexOf("_") + 1, pStr.length())));
+        }
+      }
+
+      // group partitions by status
+      Map<TaskPartitionState, Integer> statusCount = new TreeMap<TaskPartitionState, Integer>();
+      for (Integer i : partitions) {
+        TaskPartitionState s = tCtx.getPartitionState(i);
+        if (!statusCount.containsKey(s)) {
+          statusCount.put(s, 0);
+        }
+        statusCount.put(s, statusCount.get(s) + 1);
+      }
+
+      for (TaskPartitionState s : statusCount.keySet()) {
+        LOG.info(statusCount.get(s) + "/" + partitions.size() + " in state " + s.name());
+      }
+
+      LOG.info("-------");
+    }
+  }
+
+  /**
+   * Hack to invoke rebalance until bug concerning resource config changes not driving rebalance is
+   * fixed
+   */
+  public void invokeRebalance() {
+    // find a task
+    for (String resource : _admin.getResourcesInCluster(_clusterName)) {
+      IdealState is = _admin.getResourceIdealState(_clusterName, resource);
+      if (is.getStateModelDefRef().equals(TaskConstants.STATE_MODEL_NAME)) {
+        HelixDataAccessor accessor = _manager.getHelixDataAccessor();
+        accessor.updateProperty(accessor.keyBuilder().idealStates(resource), is);
+        break;
+      }
+    }
+  }
+
+  /** Constructs options set for all basic control messages */
+  private static Options constructOptions() {
+    Options options = new Options();
+    options.addOptionGroup(contructGenericRequiredOptionGroup());
+    options.addOptionGroup(constructStartOptionGroup());
+    return options;
+  }
+
+  /** Constructs option group containing options required by all drivable tasks */
+  private static OptionGroup contructGenericRequiredOptionGroup() {
+    Option zkAddressOption =
+        OptionBuilder.isRequired().hasArgs(1).withArgName("zkAddress").withLongOpt(ZK_ADDRESS)
+            .withDescription("ZK address managing target cluster").create();
+
+    Option clusterNameOption =
+        OptionBuilder.isRequired().hasArgs(1).withArgName("clusterName")
+            .withLongOpt(CLUSTER_NAME_OPTION)
+            .withDescription("Target cluster name").create();
+
+    Option taskResourceOption =
+        OptionBuilder.isRequired().hasArgs(1).withArgName("resourceName")
+            .withLongOpt(RESOURCE_OPTION)
+            .withDescription("Target workflow or task").create();
+
+    OptionGroup group = new OptionGroup();
+    group.addOption(zkAddressOption);
+    group.addOption(clusterNameOption);
+    group.addOption(taskResourceOption);
+    return group;
+  }
+
+  /** Constructs option group containing options required by all drivable tasks */
+  private static OptionGroup constructStartOptionGroup() {
+    Option workflowFileOption =
+        OptionBuilder.withLongOpt(WORKFLOW_FILE_OPTION).hasArgs(1).withArgName("workflowFile")
+            .withDescription("Local file describing workflow").create();
+
+    OptionGroup group = new OptionGroup();
+    group.addOption(workflowFileOption);
+    return group;
+  }
+
+  /** Attempts to parse options for given command, printing usage under failure */
+  private static CommandLine parseOptions(String[] args, Options options, String cmdStr) {
+    CommandLineParser cliParser = new GnuParser();
+    CommandLine cmd = null;
+
+    try {
+      cmd = cliParser.parse(options, args);
+    } catch (ParseException pe) {
+      LOG.error("CommandLineClient: failed to parse command-line options: " + pe.toString());
+      printUsage(options, cmdStr);
+      System.exit(1);
+    }
+    boolean ret = checkOptionArgsNumber(cmd.getOptions());
+    if (!ret) {
+      printUsage(options, cmdStr);
+      System.exit(1);
+    }
+
+    return cmd;
+  }
+
+  /** Ensures options argument counts are correct */
+  private static boolean checkOptionArgsNumber(Option[] options) {
+    for (Option option : options) {
+      int argNb = option.getArgs();
+      String[] args = option.getValues();
+      if (argNb == 0) {
+        if (args != null && args.length > 0) {
+          System.err.println(option.getArgName() + " shall have " + argNb + " arguments (was "
+              + Arrays.toString(args) + ")");
+          return false;
+        }
+      } else {
+        if (args == null || args.length != argNb) {
+          System.err.println(option.getArgName() + " shall have " + argNb + " arguments (was "
+              + Arrays.toString(args) + ")");
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /** Displays CLI usage for given option set and command name */
+  private static void printUsage(Options cliOptions, String cmd) {
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.setWidth(1000);
+    helpFormatter.printHelp("java " + TaskDriver.class.getName() + " " + cmd, cliOptions);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/helix-core/src/main/java/org/apache/helix/task/TaskFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskFactory.java b/helix-core/src/main/java/org/apache/helix/task/TaskFactory.java
new file mode 100644
index 0000000..0cbf24c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskFactory.java
@@ -0,0 +1,32 @@
+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.
+ */
+
+/**
+ * A factory for {@link Task} objects.
+ */
+public interface TaskFactory {
+  /**
+   * Returns a {@link Task} instance.
+   * @param config Configuration information for the task.
+   * @return A {@link Task} instance.
+   */
+  Task createNewTask(String config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/80fc2be5/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
new file mode 100644
index 0000000..d41668d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskPartitionState.java
@@ -0,0 +1,42 @@
+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.
+ */
+
+/**
+ * Enumeration of the states in the "Task" state model.
+ */
+public enum TaskPartitionState {
+  /** The initial state of the state model. */
+  INIT,
+  /** Indicates that the task is currently running. */
+  RUNNING,
+  /** Indicates that the task was stopped by the controller. */
+  STOPPED,
+  /** Indicates that the task completed normally. */
+  COMPLETED,
+  /** Indicates that the task timed out. */
+  TIMED_OUT,
+  /** Indicates an error occurred during task execution. */
+  TASK_ERROR,
+  /** Helix's own internal error state. */
+  ERROR,
+  /** A Helix internal state. */
+  DROPPED
+}