Job Config and logic refactoring

1. Support identical task initialization with job command and number of tasks
2. Remove unused MaxForcedReassignmentPerTask field
3. Refactor logics of failure.


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

Branch: refs/heads/helix-0.6.x
Commit: 9fc6c540bbcb4d7c71f0b7fe89e2acbc5955e859
Parents: 7bb2a9d
Author: Junkai Xue <j...@linkedin.com>
Authored: Tue Sep 13 16:01:39 2016 -0700
Committer: Junkai Xue <j...@linkedin.com>
Committed: Wed Sep 21 10:43:35 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/task/JobConfig.java   | 52 +++++++++++++++-----
 .../org/apache/helix/task/JobRebalancer.java    | 19 +------
 .../java/org/apache/helix/task/TaskConfig.java  | 50 +++++++++----------
 .../org/apache/helix/task/beans/JobBean.java    |  2 +-
 .../org/apache/helix/task/beans/TaskBean.java   |  1 +
 .../TestGenericTaskAssignmentCalculator.java    |  4 +-
 .../task/TestIndependentTaskRebalancer.java     | 45 +++--------------
 .../integration/task/TestUserContentStore.java  |  6 +--
 8 files changed, 81 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/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 7a4e2d3..a966f35 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
@@ -83,6 +83,7 @@ public class JobConfig {
      * The maximum number of times the task rebalancer may attempt to execute 
a task.
      */
     MaxAttemptsPerTask,
+    @Deprecated
     /**
      * The maximum number of times Helix will intentionally move a failing task
      */
@@ -134,6 +135,7 @@ public class JobConfig {
   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;
+  public static final int DEFAULT_NUMBER_OF_TASKS = 0;
 
   private final String _workflow;
   private final String _targetResource;
@@ -218,10 +220,6 @@ public class JobConfig {
     return _maxAttemptsPerTask;
   }
 
-  public int getMaxForcedReassignmentsPerTask() {
-    return _maxForcedReassignmentsPerTask;
-  }
-
   public int getFailureThreshold() {
     return _failureThreshold;
   }
@@ -308,6 +306,8 @@ public class JobConfig {
    * A builder for {@link JobConfig}. Validates the configurations.
    */
   public static class Builder {
+    private final String NUMBER_OF_TASKS = "NumberOfTasks";
+
     private String _workflow;
     private String _targetResource;
     private String _jobType;
@@ -325,10 +325,18 @@ public class JobConfig {
     private long _retryDelay = DEFAULT_TASK_RETRY_DELAY;
     private boolean _disableExternalView = DEFAULT_DISABLE_EXTERNALVIEW;
     private boolean _ignoreDependentJobFailure = 
DEFAULT_IGNORE_DEPENDENT_JOB_FAILURE;
+    private int _numberOfTasks = DEFAULT_NUMBER_OF_TASKS;
 
     public JobConfig build() {
       validate();
 
+      if (_taskConfigMap.isEmpty()) {
+        for (int i = 0; i < _numberOfTasks; i++) {
+          TaskConfig taskConfig = new TaskConfig(null, null);
+          _taskConfigMap.put(taskConfig.getId(), taskConfig);
+        }
+      }
+
       return new JobConfig(_workflow, _targetResource, _targetPartitions, 
_targetPartitionStates,
           _command, _commandConfig, _timeoutPerTask, 
_numConcurrentTasksPerInstance,
           _maxAttemptsPerTask, _maxForcedReassignmentsPerTask, 
_failureThreshold, _retryDelay,
@@ -376,10 +384,6 @@ public class JobConfig {
         b.setMaxAttemptsPerTask(
             
Integer.parseInt(cfg.get(JobConfigProperty.MaxAttemptsPerTask.name())));
       }
-      if 
(cfg.containsKey(JobConfigProperty.MaxForcedReassignmentsPerTask.name())) {
-        b.setMaxForcedReassignmentsPerTask(
-            
Integer.parseInt(cfg.get(JobConfigProperty.MaxForcedReassignmentsPerTask.name())));
-      }
       if (cfg.containsKey(JobConfigProperty.FailureThreshold.name())) {
         b.setFailureThreshold(
             
Integer.parseInt(cfg.get(JobConfigProperty.FailureThreshold.name())));
@@ -429,6 +433,11 @@ public class JobConfig {
       return this;
     }
 
+    public Builder setNumberOfTasks(int v) {
+      _numberOfTasks = v;
+      return this;
+    }
+
     public Builder setJobCommandConfigMap(Map<String, String> v) {
       _commandConfig = v;
       return this;
@@ -449,6 +458,8 @@ public class JobConfig {
       return this;
     }
 
+    // This field will be ignored by Helix
+    @Deprecated
     public Builder setMaxForcedReassignmentsPerTask(int v) {
       _maxForcedReassignmentsPerTask = v;
       return this;
@@ -508,9 +519,25 @@ public class JobConfig {
         throw new IllegalArgumentException(
             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));
+      if (_taskConfigMap.isEmpty()) {
+        // Check Job command is not null when none taskconfig specified
+        if (_command == null) {
+          throw new IllegalArgumentException(
+              String.format("%s cannot be null", JobConfigProperty.Command));
+        }
+        // Check number of task is set when Job command is not null and none 
taskconfig specified
+        if (_targetResource == null && _numberOfTasks == 0) {
+          throw new IllegalArgumentException("Either targetResource or 
numberOfTask should be set");
+        }
+      }
+      // Check each either Job command is not null or none of task command is 
not null
+      if (_command == null) {
+        for (TaskConfig taskConfig : _taskConfigMap.values()) {
+          if (taskConfig.getCommand() == null) {
+            throw new IllegalArgumentException(
+                String.format("Task % command cannot be null", 
taskConfig.getId()));
+          }
+        }
       }
       if (_timeoutPerTask < 0) {
         throw new IllegalArgumentException(String
@@ -547,12 +574,11 @@ public class JobConfig {
       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);
+          
.setIgnoreDependentJobFailure(jobBean.ignoreDependentJobFailure).setNumberOfTasks(jobBean.numberOfTasks);
 
       if (jobBean.jobCommandConfigMap != null) {
         b.setJobCommandConfigMap(jobBean.jobCommandConfigMap);

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/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 cf7f5e6..7676dab 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
@@ -345,24 +345,9 @@ public class JobRebalancer extends TaskRebalancer {
           // 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;
-            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
+            // 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) {
+            if (skippedPartitions.size() >= jobCfg.getFailureThreshold()) {
               markJobFailed(jobResource, jobCtx, workflowConfig, workflowCtx);
               markAllPartitionsError(jobCtx, currState, false);
               addAllPartitions(allPartitions, partitionsToDropFromIs);

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/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 b990f99..621d371 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
@@ -36,6 +36,7 @@ public class TaskConfig {
   private enum TaskConfigProperty {
     TASK_ID,
     TASK_COMMAND,
+    @Deprecated
     TASK_SUCCESS_OPTIONAL,
     TASK_TARGET_PARTITION
   }
@@ -44,18 +45,26 @@ public class TaskConfig {
 
   private final Map<String, String> _configMap;
 
+  @Deprecated
+  public TaskConfig(String command, Map<String, String> configMap, boolean 
successOptional,
+      String id, String target) {
+    this(command, configMap, id, target);
+  }
+
+  @Deprecated
+  public TaskConfig(String command, Map<String, String> configMap, boolean 
successOptional) {
+    this(command, configMap, null, null);
+  }
+
   /**
    * 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 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
    */
-  public TaskConfig(String command, Map<String, String> configMap, boolean 
successOptional,
-      String id, String target) {
+  public TaskConfig(String command, Map<String, String> configMap, String id, 
String target) {
     if (configMap == null) {
       configMap = Maps.newHashMap();
     }
@@ -65,8 +74,6 @@ public class TaskConfig {
     if (command != null) {
       configMap.put(TaskConfigProperty.TASK_COMMAND.name(), command);
     }
-    configMap
-        .put(TaskConfigProperty.TASK_SUCCESS_OPTIONAL.name(), 
Boolean.toString(successOptional));
     configMap.put(TaskConfigProperty.TASK_ID.name(), id);
     if (target != null) {
       configMap.put(TaskConfigProperty.TASK_TARGET_PARTITION.name(), target);
@@ -79,11 +86,9 @@ public class TaskConfig {
    *
    * @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
    */
-  public TaskConfig(String command, Map<String, String> configMap, boolean 
successOptional) {
-    this(command, configMap, successOptional, null, null);
+  public TaskConfig(String command, Map<String, String> configMap) {
+    this(command, configMap, null, null);
   }
 
   /**
@@ -115,16 +120,13 @@ public class TaskConfig {
 
   /**
    * Check if this task must succeed for a job to succeed
-   *
+   * This field has been ignored by Helix
    * @return true if success is optional, false otherwise
    */
+  @Deprecated
   public boolean isSuccessOptional() {
-    String successOptionalStr = 
_configMap.get(TaskConfigProperty.TASK_SUCCESS_OPTIONAL.name());
-    if (successOptionalStr == null) {
-      return false;
-    } else {
-      return Boolean.parseBoolean(successOptionalStr);
-    }
+    // This option will not be used in rebalancer anymore, deprecate it.
+    return true;
   }
 
   /**
@@ -154,7 +156,7 @@ public class TaskConfig {
     private Map<String, String> _configMap;
 
     public TaskConfig build() {
-      return new TaskConfig(_command, _configMap, _successOptional, _taskId, 
_targetPartition);
+      return new TaskConfig(_command, _configMap, _taskId, _targetPartition);
     }
 
     public String getTaskId() {
@@ -184,10 +186,12 @@ public class TaskConfig {
       return this;
     }
 
+    @Deprecated
     public boolean isSuccessOptional() {
       return _successOptional;
     }
 
+    @Deprecated
     public Builder setSuccessOptional(boolean successOptional) {
       _successOptional = successOptional;
       return this;
@@ -208,7 +212,7 @@ public class TaskConfig {
      * @return instantiated TaskConfig
      */
     public static TaskConfig from(String target) {
-      return new TaskConfig(null, null, false, null, target);
+      return new TaskConfig(null, null, null, target);
     }
 
     /**
@@ -218,7 +222,7 @@ public class TaskConfig {
      * @return instantiated TaskConfig
      */
     public static TaskConfig from(TaskBean bean) {
-      return new TaskConfig(bean.command, bean.taskConfigMap, 
bean.successOptional);
+      return new TaskConfig(bean.command, bean.taskConfigMap);
     }
 
     /**
@@ -232,11 +236,7 @@ public class TaskConfig {
       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);
+      return new TaskConfig(command, rawConfigMap, taskId, targetPartition);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/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 dd7ebab..9a376f8 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
@@ -41,9 +41,9 @@ public class JobBean {
   public long timeoutPerPartition = JobConfig.DEFAULT_TIMEOUT_PER_TASK;
   public int numConcurrentTasksPerInstance = 
JobConfig.DEFAULT_NUM_CONCURRENT_TASKS_PER_INSTANCE;
   public int maxAttemptsPerTask = JobConfig.DEFAULT_MAX_ATTEMPTS_PER_TASK;
-  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;
+  public int numberOfTasks = JobConfig.DEFAULT_NUMBER_OF_TASKS;
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/helix-core/src/main/java/org/apache/helix/task/beans/TaskBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/beans/TaskBean.java 
b/helix-core/src/main/java/org/apache/helix/task/beans/TaskBean.java
index 97ecfc0..a61556b 100644
--- a/helix-core/src/main/java/org/apache/helix/task/beans/TaskBean.java
+++ b/helix-core/src/main/java/org/apache/helix/task/beans/TaskBean.java
@@ -29,5 +29,6 @@ import java.util.Map;
 public class TaskBean {
   public String command;
   public Map<String, String> taskConfigMap;
+  @Deprecated
   public boolean successOptional = false;
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericTaskAssignmentCalculator.java
----------------------------------------------------------------------
diff --git 
a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericTaskAssignmentCalculator.java
 
b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericTaskAssignmentCalculator.java
index 0410db2..5645009 100644
--- 
a/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericTaskAssignmentCalculator.java
+++ 
b/helix-core/src/test/java/org/apache/helix/integration/task/TestGenericTaskAssignmentCalculator.java
@@ -104,7 +104,7 @@ public class TestGenericTaskAssignmentCalculator extends 
TaskTestBase {
     _driver = new TaskDriver(_manager);
 
     Map<String, String> taskConfigMap = Maps.newHashMap();
-    _taskConfig = new TaskConfig("TaskOne", taskConfigMap, false);
+    _taskConfig = new TaskConfig("TaskOne", taskConfigMap);
     _jobCommandMap = Maps.newHashMap();
   }
 
@@ -136,7 +136,7 @@ public class TestGenericTaskAssignmentCalculator extends 
TaskTestBase {
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(20);
     for (int i = 0; i < 50; i++) {
       Map<String, String> taskConfigMap = Maps.newHashMap();
-      taskConfigs.add(new TaskConfig("TaskOne", taskConfigMap, false));
+      taskConfigs.add(new TaskConfig("TaskOne", taskConfigMap));
     }
     JobConfig.Builder jobBuilder =
         new 
JobConfig.Builder().setCommand("DummyCommand").setJobCommandConfigMap(_jobCommandMap)

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/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 c4d588c..64b9073 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
@@ -136,8 +136,8 @@ public class TestIndependentTaskRebalancer extends 
TaskTestBase {
     String jobName = TestHelper.getTestMethodName();
     Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(2);
-    TaskConfig taskConfig1 = new TaskConfig("TaskOne", null, true);
-    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", null, true);
+    TaskConfig taskConfig1 = new TaskConfig("TaskOne", null);
+    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", null);
     taskConfigs.add(taskConfig1);
     taskConfigs.add(taskConfig2);
     Map<String, String> jobCommandMap = Maps.newHashMap();
@@ -164,8 +164,8 @@ public class TestIndependentTaskRebalancer extends 
TaskTestBase {
     Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(2);
     Map<String, String> taskConfigMap = 
Maps.newHashMap(ImmutableMap.of("fail", "" + true));
-    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap, false);
-    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", null, false);
+    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap);
+    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", null);
     taskConfigs.add(taskConfig1);
     taskConfigs.add(taskConfig2);
     Map<String, String> jobConfigMap = Maps.newHashMap();
@@ -185,35 +185,6 @@ public class TestIndependentTaskRebalancer extends 
TaskTestBase {
     Assert.assertTrue(_invokedClasses.contains(TaskTwo.class.getName()));
   }
 
-  @Test public void testOptionalTaskFailure() throws Exception {
-    // Create a job with two different tasks
-    String jobName = TestHelper.getTestMethodName();
-    Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
-    List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(2);
-    Map<String, String> taskConfigMap = 
Maps.newHashMap(ImmutableMap.of("fail", "" + true));
-    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap, true);
-    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", null, false);
-    taskConfigs.add(taskConfig1);
-    taskConfigs.add(taskConfig2);
-    Map<String, String> jobCommandMap = Maps.newHashMap();
-    jobCommandMap.put("Timeout", "1000");
-
-    JobConfig.Builder jobBuilder =
-        new 
JobConfig.Builder().setCommand("DummyCommand").addTaskConfigs(taskConfigs)
-            .setJobCommandConfigMap(jobCommandMap);
-    workflowBuilder.addJob(jobName, jobBuilder);
-
-    _driver.start(workflowBuilder.build());
-
-    // Ensure the job completes
-    _driver.pollForWorkflowState(jobName, TaskState.IN_PROGRESS);
-    _driver.pollForWorkflowState(jobName, TaskState.COMPLETED);
-
-    // Ensure that each class was invoked
-    Assert.assertTrue(_invokedClasses.contains(TaskOne.class.getName()));
-    Assert.assertTrue(_invokedClasses.contains(TaskTwo.class.getName()));
-  }
-
   @Test public void testReassignment() throws Exception {
     final int NUM_INSTANCES = 5;
     String jobName = TestHelper.getTestMethodName();
@@ -221,13 +192,13 @@ public class TestIndependentTaskRebalancer extends 
TaskTestBase {
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(2);
     Map<String, String> taskConfigMap = Maps.newHashMap(ImmutableMap
         .of("fail", "" + true, "failInstance", PARTICIPANT_PREFIX + '_' + 
(_startPort + 1)));
-    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap, false);
+    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap);
     taskConfigs.add(taskConfig1);
     Map<String, String> jobCommandMap = Maps.newHashMap();
     jobCommandMap.put("Timeout", "1000");
 
     JobConfig.Builder jobBuilder = new 
JobConfig.Builder().setCommand("DummyCommand")
-        .setMaxForcedReassignmentsPerTask(NUM_INSTANCES - 
1).addTaskConfigs(taskConfigs)
+        .addTaskConfigs(taskConfigs)
         .setJobCommandConfigMap(jobCommandMap);
     workflowBuilder.addJob(jobName, jobBuilder);
 
@@ -254,7 +225,7 @@ public class TestIndependentTaskRebalancer extends 
TaskTestBase {
     Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(1);
     Map<String, String> taskConfigMap = Maps.newHashMap();
-    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap, false);
+    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap);
     taskConfigs.add(taskConfig1);
     Map<String, String> jobCommandMap = Maps.newHashMap();
     jobCommandMap.put("Timeout", "1000");
@@ -289,7 +260,7 @@ public class TestIndependentTaskRebalancer extends 
TaskTestBase {
     Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(1);
     Map<String, String> taskConfigMap = Maps.newHashMap();
-    TaskConfig taskConfig1 = new TaskConfig("SingleFailTask", taskConfigMap, 
false);
+    TaskConfig taskConfig1 = new TaskConfig("SingleFailTask", taskConfigMap);
     taskConfigs.add(taskConfig1);
     Map<String, String> jobCommandMap = Maps.newHashMap();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/9fc6c540/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
----------------------------------------------------------------------
diff --git 
a/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
 
b/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
index b2b27ef..13cd531 100644
--- 
a/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
+++ 
b/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
@@ -123,7 +123,7 @@ public class TestUserContentStore extends TaskTestBase {
     Workflow.Builder workflowBuilder = new Workflow.Builder(jobName);
     List<TaskConfig> taskConfigs = Lists.newArrayListWithCapacity(1);
     Map<String, String> taskConfigMap = Maps.newHashMap();
-    TaskConfig taskConfig1 = new TaskConfig("ContentStoreTask", taskConfigMap, 
false);
+    TaskConfig taskConfig1 = new TaskConfig("ContentStoreTask", taskConfigMap);
     taskConfigs.add(taskConfig1);
     Map<String, String> jobCommandMap = Maps.newHashMap();
     jobCommandMap.put("Timeout", "1000");
@@ -148,8 +148,8 @@ public class TestUserContentStore extends TaskTestBase {
     List<TaskConfig> taskConfigs2 = Lists.newArrayListWithCapacity(1);
     Map<String, String> taskConfigMap1 = Maps.newHashMap();
     Map<String, String> taskConfigMap2 = Maps.newHashMap();
-    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap1, false);
-    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", taskConfigMap2, false);
+    TaskConfig taskConfig1 = new TaskConfig("TaskOne", taskConfigMap1);
+    TaskConfig taskConfig2 = new TaskConfig("TaskTwo", taskConfigMap2);
 
     taskConfigs1.add(taskConfig1);
     taskConfigs2.add(taskConfig2);

Reply via email to