You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ji...@apache.org on 2018/08/07 06:59:46 UTC

[incubator-druid] branch master updated: Native parallel batch indexing without shuffle (#5492)

This is an automated email from the ASF dual-hosted git repository.

jihoonson pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new 56ab436  Native parallel batch indexing without shuffle (#5492)
56ab436 is described below

commit 56ab4363ea30ead876c11b43e1480ff6c0dfd941
Author: Jihoon Son <ji...@apache.org>
AuthorDate: Mon Aug 6 23:59:42 2018 -0700

    Native parallel batch indexing without shuffle (#5492)
    
    * Native parallel indexing without shuffle
    
    * fix build
    
    * fix ci
    
    * fix ingestion without intervals
    
    * fix retry
    
    * fix retry
    
    * add it test
    
    * use chat handler
    
    * fix build
    
    * add docs
    
    * fix ITUnionQueryTest
    
    * fix failures
    
    * disable metrics reporting
    
    * working
    
    * Fix split of static-s3 firehose
    
    * Add endpoints to supervisor task and a unit test for endpoints
    
    * increase timeout in test
    
    * Added doc
    
    * Address comments
    
    * Fix overlapping locks
    
    * address comments
    
    * Fix static s3 firehose
    
    * Fix test
    
    * fix build
    
    * fix test
    
    * fix typo in docs
    
    * add missing maxBytesInMemory to doc
    
    * address comments
    
    * fix race in test
    
    * fix test
    
    * Rename to ParallelIndexSupervisorTask
    
    * fix teamcity
    
    * address comments
    
    * Fix license
    
    * addressing comments
    
    * addressing comments
    
    * indexTaskClient-based segmentAllocator instead of CountingActionBasedSegmentAllocator
    
    * Fix race in TaskMonitor and move HTTP endpoints to supervisorTask from runner
    
    * Add more javadocs
    
    * use StringUtils.nonStrictFormat for logging
    
    * fix typo and remove unused class
    
    * fix tests
    
    * change package
    
    * fix strict build
    
    * tmp
    
    * Fix overlord api according to the recent change in master
    
    * Fix it test
---
 .../io/druid/data/input/FiniteFirehoseFactory.java |  68 ++
 .../java/io/druid/data/input/FirehoseFactory.java  |   5 +
 .../main/java/io/druid/data/input/InputSplit.java  |  35 +-
 .../impl/AbstractTextFilesFirehoseFactory.java     |  40 +-
 .../java/io/druid/indexer/RunnerTaskState.java     |   2 +-
 .../main/java/io/druid/indexer/TaskStatusPlus.java |  16 +
 .../PrefetchableTextFilesFirehoseFactoryTest.java  |   8 +
 .../metadata/MetadataStorageActionHandler.java     |   3 +
 .../development/extensions-contrib/azure.md        |   3 +
 .../development/extensions-contrib/cloudfiles.md   |   3 +
 .../development/extensions-contrib/google.md       |   3 +
 docs/content/development/extensions-core/s3.md     |   2 +
 docs/content/ingestion/firehose.md                 |   4 +
 docs/content/ingestion/native_tasks.md             | 530 +++++++++++++++
 docs/content/ingestion/tasks.md                    | 225 ++-----
 .../azure/StaticAzureBlobStoreFirehoseFactory.java |  18 +
 .../StaticCloudFilesFirehoseFactory.java           |  18 +
 .../StaticGoogleBlobStoreFirehoseFactory.java      |  18 +
 .../io/druid/indexing/kafka/KafkaIndexTask.java    |  15 +-
 .../druid/indexing/kafka/KafkaIndexTaskClient.java | 437 ++-----------
 .../kafka/KafkaIndexTaskClientFactory.java         |   6 +-
 .../druid/indexing/kafka/KafkaIndexTaskModule.java |   8 +
 .../indexing/kafka/KafkaIndexTaskClientTest.java   |   8 +-
 .../druid/indexing/kafka/KafkaIndexTaskTest.java   |   5 +-
 .../druid/firehose/s3/StaticS3FirehoseFactory.java |  81 ++-
 .../firehose/s3/StaticS3FirehoseFactoryTest.java   |  67 +-
 .../appenderator/ActionBasedSegmentAllocator.java  |  16 +-
 .../SegmentAllocateActionGenerator.java            |  44 ++
 .../java/io/druid/indexing/common/Counters.java    |  72 ++
 .../io/druid/indexing/common/IndexTaskClient.java  | 381 +++++++++++
 .../indexing/common/actions/SurrogateAction.java   |  90 +++
 .../druid/indexing/common/actions/TaskAction.java  |   6 +-
 .../common/actions/TaskActionPreconditions.java    |   7 +-
 .../indexing/common/actions/TaskActionToolbox.java |  22 +-
 .../druid/indexing/common/task/AbstractTask.java   |   4 +-
 .../task/AppenderatorDriverRealtimeIndexTask.java  |  15 +-
 .../common/task/ClientBasedTaskInfoProvider.java   |  55 ++
 .../io/druid/indexing/common/task/IndexTask.java   |  22 +-
 .../common/task/IndexTaskClientFactory.java        |  16 +-
 .../java/io/druid/indexing/common/task/Task.java   |   4 +
 .../task/batch/parallel/ParallelIndexIOConfig.java |  25 +-
 .../batch/parallel/ParallelIndexIngestionSpec.java |  67 ++
 .../task/batch/parallel/ParallelIndexSubTask.java  | 437 +++++++++++++
 .../batch/parallel/ParallelIndexSubTaskSpec.java   |  67 ++
 .../parallel/ParallelIndexSupervisorTask.java      | 558 ++++++++++++++++
 .../batch/parallel/ParallelIndexTaskClient.java    | 110 ++++
 .../parallel/ParallelIndexTaskClientFactory.java   |  29 +-
 .../batch/parallel/ParallelIndexTaskRunner.java    | 137 ++++
 .../batch/parallel/ParallelIndexTuningConfig.java  | 195 ++++++
 .../batch/parallel/ParallelIndexingProgress.java   |  12 +-
 .../task/batch/parallel/PushedSegmentsReport.java  |  60 ++
 .../SinglePhaseParallelIndexTaskRunner.java        | 484 ++++++++++++++
 .../SinglePhaseParallelIndexingProgress.java       | 117 ++++
 .../common/task/batch/parallel/SubTaskSpec.java    |  84 +++
 .../common/task/batch/parallel/TaskHistory.java    |  66 ++
 .../common/task/batch/parallel/TaskMonitor.java    | 477 ++++++++++++++
 .../druid/indexing/overlord/ForkingTaskRunner.java |  20 +
 .../indexing/overlord/HeapMemoryTaskStorage.java   |  26 +
 .../indexing/overlord/MetadataTaskStorage.java     |   7 +
 .../druid/indexing/overlord/RemoteTaskRunner.java  |  19 +
 .../io/druid/indexing/overlord/TaskLockbox.java    |  16 +-
 .../io/druid/indexing/overlord/TaskRunner.java     |   8 +
 .../io/druid/indexing/overlord/TaskStorage.java    |  14 +-
 .../indexing/overlord/TaskStorageQueryAdapter.java |   6 +
 .../overlord/hrtr/HttpRemoteTaskRunner.java        |  22 +
 .../indexing/overlord/http/OverlordResource.java   |  94 ++-
 .../indexing/overlord/http/TaskStatusResponse.java |  12 +-
 .../java/io/druid/indexing/common/TestUtils.java   |   7 +-
 .../common/actions/SegmentInsertActionTest.java    |   2 +-
 .../SegmentTransactionalInsertActionTest.java      |   2 +-
 .../common/actions/SurrogateActionTest.java        |  45 ++
 .../indexing/common/actions/TaskActionTestKit.java |   5 +-
 .../AppenderatorDriverRealtimeIndexTaskTest.java   |   5 +-
 .../indexing/common/task/IngestionTestBase.java    | 108 +++
 .../common/task/RealtimeIndexTaskTest.java         |   5 +-
 .../AbstractParallelIndexSupervisorTaskTest.java   | 399 +++++++++++
 .../ParallelIndexSupervisorTaskResourceTest.java   | 727 +++++++++++++++++++++
 .../ParallelIndexSupervisorTaskSerdeTest.java      | 164 +++++
 .../parallel/ParallelIndexSupervisorTaskTest.java  | 370 +++++++++++
 .../task/batch/parallel/TaskMonitorTest.java       | 220 +++++++
 .../firehose/IngestSegmentFirehoseFactoryTest.java |  10 +-
 .../druid/indexing/overlord/TaskLifecycleTest.java |  14 +-
 .../overlord/http/OverlordResourceTest.java        |  35 +-
 .../druid/indexing/overlord/http/OverlordTest.java |   4 +-
 .../clients/OverlordResourceTestClient.java        |  12 +-
 .../tests/indexer/AbstractITBatchIndexTest.java    |  91 +++
 .../java/io/druid/tests/indexer/ITIndexerTest.java |  78 +--
 .../druid/tests/indexer/ITParallelIndexTest.java   |  37 +-
 .../io/druid/tests/indexer/ITUnionQueryTest.java   |   6 +-
 .../data/batch_index/wikipedia_index_data1.json    |   3 +
 .../data/batch_index/wikipedia_index_data2.json    |   3 +
 .../data/batch_index/wikipedia_index_data3.json    |   4 +
 .../union_query}/wikipedia_index_data.json         |   0
 .../resources/indexer/wikipedia_index_task.json    |   4 +-
 .../indexer/wikipedia_parallel_index_queries.json  |  71 ++
 ...ask.json => wikipedia_parallel_index_task.json} |  13 +-
 ...eClient.java => HttpIndexingServiceClient.java} |  85 ++-
 .../client/indexing/IndexingServiceClient.java     | 228 +------
 .../java/io/druid/client/indexing/TaskStatus.java  | 101 +++
 .../druid/client/indexing}/TaskStatusResponse.java |  17 +-
 .../java/io/druid/guice/http/HttpClientModule.java |   6 +-
 .../IndexerMetadataStorageCoordinator.java         |   6 +-
 .../overlord/supervisor/SupervisorSpec.java        |   6 +
 .../IndexerSQLMetadataStorageCoordinator.java      |   9 -
 .../metadata/SQLMetadataStorageActionHandler.java  |  43 +-
 .../java/io/druid/segment/indexing/IOConfig.java   |   2 +-
 .../segment/realtime/firehose/ChatHandlers.java    |  59 ++
 .../realtime/firehose/HttpFirehoseFactory.java     |  22 +
 .../realtime/firehose/LocalFirehoseFactory.java    |  11 +
 .../timeline/partition/NumberedShardSpec.java      |  25 +
 .../client/indexing/NoopIndexingServiceClient.java | 116 ++++
 .../AsyncManagementForwardingServletTest.java      |   6 +-
 .../DruidCoordinatorSegmentMergerTest.java         |   5 +-
 .../DruidCoordinatorSegmentCompactorTest.java      |   3 +-
 .../src/main/java/io/druid/cli/CliCoordinator.java |   3 +-
 .../main/java/io/druid/cli/CliMiddleManager.java   |   7 +
 .../src/main/java/io/druid/cli/CliOverlord.java    |   8 +-
 services/src/main/java/io/druid/cli/CliPeon.java   |  13 +-
 118 files changed, 7735 insertions(+), 1041 deletions(-)

diff --git a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java
new file mode 100644
index 0000000..bb62161
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.data.input;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import io.druid.data.input.impl.InputRowParser;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+/**
+ * {@link FiniteFirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is
+ * limited.
+ *
+ * @param <T> parser type
+ * @param <S> input split type
+ */
+public interface FiniteFirehoseFactory<T extends InputRowParser, S> extends FirehoseFactory<T>
+{
+  /**
+   * Returns true if this {@link FiniteFirehoseFactory} supports parallel batch indexing.
+   */
+  @JsonIgnore
+  @Override
+  default boolean isSplittable()
+  {
+    return true;
+  }
+
+  /**
+   * Returns a {@link Stream} for {@link InputSplit}s. In parallel batch indexing, each {@link InputSplit} is processed
+   * by a sub task.
+   *
+   * Listing splits may cause high overhead in some implementations. In this case, {@link InputSplit}s should be listed
+   * lazily so that the listing overhead could be amortized.
+   */
+  @JsonIgnore
+  Stream<InputSplit<S>> getSplits() throws IOException;
+
+  /**
+   * Returns number of splits returned by {@link #getSplits()}.
+   */
+  @JsonIgnore
+  int getNumSplits() throws IOException;
+
+  /**
+   * Returns the same {@link FiniteFirehoseFactory} but with the given {@link InputSplit}. The returned
+   * {@link FiniteFirehoseFactory} is used by sub tasks in parallel batch indexing.
+   */
+  FiniteFirehoseFactory<T, S> withSplit(InputSplit<S> split);
+}
diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java
index 2396769..d5c4548 100644
--- a/api/src/main/java/io/druid/data/input/FirehoseFactory.java
+++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java
@@ -72,4 +72,9 @@ public interface FirehoseFactory<T extends InputRowParser>
   {
     return connect(parser);
   }
+
+  default boolean isSplittable()
+  {
+    return false;
+  }
 }
diff --git a/server/src/main/java/io/druid/segment/indexing/IOConfig.java b/api/src/main/java/io/druid/data/input/InputSplit.java
similarity index 54%
copy from server/src/main/java/io/druid/segment/indexing/IOConfig.java
copy to api/src/main/java/io/druid/data/input/InputSplit.java
index a278847..b3886b6 100644
--- a/server/src/main/java/io/druid/segment/indexing/IOConfig.java
+++ b/api/src/main/java/io/druid/data/input/InputSplit.java
@@ -17,17 +17,36 @@
  * under the License.
  */
 
-package io.druid.segment.indexing;
+package io.druid.data.input;
 
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
 
 /**
+ * Input unit for distributed batch ingestion. Used in {@link FiniteFirehoseFactory}.
+ * An {@link InputSplit} represents the input data processed by a {@code io.druid.indexing.common.task.Task}.
  */
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
-@JsonSubTypes(value = {
-    @JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class),
-})
-public interface IOConfig
+public class InputSplit<T>
 {
+  private final T split;
+
+  @JsonCreator
+  public InputSplit(@JsonProperty("split") T split)
+  {
+    this.split = split;
+  }
+
+  @JsonProperty("split")
+  public T get()
+  {
+    return split;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "InputSplit{" +
+           "split=" + split +
+           "}";
+  }
 }
diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java
index c9340e6..6c1ce34 100644
--- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java
+++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java
@@ -22,20 +22,22 @@ package io.druid.data.input.impl;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
+import io.druid.data.input.FiniteFirehoseFactory;
 import io.druid.data.input.Firehose;
-import io.druid.data.input.FirehoseFactory;
+import io.druid.data.input.InputSplit;
 import io.druid.java.util.common.logger.Logger;
-import org.apache.commons.io.Charsets;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.LineIterator;
 
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
+import java.util.stream.Stream;
 
 /**
  * This is an abstract class for firehose factory for making firehoses reading text files.
@@ -44,7 +46,7 @@ import java.util.NoSuchElementException;
  * @param <T> object type representing input data
  */
 public abstract class AbstractTextFilesFirehoseFactory<T>
-    implements FirehoseFactory<StringInputRowParser>
+    implements FiniteFirehoseFactory<StringInputRowParser, T>
 {
   private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class);
 
@@ -53,9 +55,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
   @Override
   public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException
   {
-    if (objects == null) {
-      objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects"));
-    }
+    initializeObjectsIfNeeded();
     final Iterator<T> iterator = objects.iterator();
     return new FileIteratingFirehose(
         new Iterator<LineIterator>()
@@ -74,7 +74,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
             }
             final T object = iterator.next();
             try {
-              return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), Charsets.UTF_8);
+              return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), StandardCharsets.UTF_8);
             }
             catch (Exception e) {
               LOG.error(
@@ -90,6 +90,32 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
     );
   }
 
+  protected void initializeObjectsIfNeeded() throws IOException
+  {
+    if (objects == null) {
+      objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects"));
+    }
+  }
+
+  public List<T> getObjects()
+  {
+    return objects;
+  }
+
+  @Override
+  public Stream<InputSplit<T>> getSplits() throws IOException
+  {
+    initializeObjectsIfNeeded();
+    return getObjects().stream().map(InputSplit::new);
+  }
+
+  @Override
+  public int getNumSplits() throws IOException
+  {
+    initializeObjectsIfNeeded();
+    return getObjects().size();
+  }
+
   /**
    * Initialize objects to be read by this firehose.  Since firehose factories are constructed whenever
    * io.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred
diff --git a/api/src/main/java/io/druid/indexer/RunnerTaskState.java b/api/src/main/java/io/druid/indexer/RunnerTaskState.java
index a834491..11efe7d 100644
--- a/api/src/main/java/io/druid/indexer/RunnerTaskState.java
+++ b/api/src/main/java/io/druid/indexer/RunnerTaskState.java
@@ -25,5 +25,5 @@ public enum RunnerTaskState
   WAITING,
   PENDING,
   RUNNING,
-  NONE; // is used for a completed task
+  NONE // is used for a completed task
 }
diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java
index 71f0ac1..2f76281 100644
--- a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java
+++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java
@@ -172,4 +172,20 @@ public class TaskStatusPlus
         getErrorMsg()
     );
   }
+
+  @Override
+  public String toString()
+  {
+    return "TaskStatusPlus{" +
+           "id='" + id + '\'' +
+           ", type='" + type + '\'' +
+           ", createdTime=" + createdTime +
+           ", queueInsertionTime=" + queueInsertionTime +
+           ", state=" + state +
+           ", duration=" + duration +
+           ", location=" + location +
+           ", dataSource='" + dataSource + '\'' +
+           ", errorMsg='" + errorMsg + '\'' +
+           '}';
+  }
 }
diff --git a/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java
index 3bfb465..2e0f0dc 100644
--- a/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java
+++ b/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java
@@ -23,7 +23,9 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Lists;
 import com.google.common.io.CountingOutputStream;
+import io.druid.data.input.FiniteFirehoseFactory;
 import io.druid.data.input.Firehose;
+import io.druid.data.input.InputSplit;
 import io.druid.data.input.Row;
 import io.druid.data.input.impl.CSVParseSpec;
 import io.druid.data.input.impl.DimensionsSpec;
@@ -605,6 +607,12 @@ public class PrefetchableTextFilesFirehoseFactoryTest
     private int readCount;
     private int numConnectionResets;
 
+    @Override
+    public FiniteFirehoseFactory<StringInputRowParser, File> withSplit(InputSplit<File> split)
+    {
+      throw new UnsupportedOperationException();
+    }
+
     private class TestInputStream extends InputStream
     {
       private static final int NUM_READ_COUNTS_BEFORE_ERROR = 10;
diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java
index 0fc4312..9661a64 100644
--- a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java
+++ b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java
@@ -78,6 +78,9 @@ public interface MetadataStorageActionHandler<EntryType, StatusType, LogType, Lo
    */
   Optional<StatusType> getStatus(String entryId);
 
+  @Nullable
+  TaskInfo<EntryType, StatusType> getTaskInfo(String entryId);
+
   /**
    * Return up to {@code maxNumStatuses} {@link TaskInfo} objects for all inactive entries
    * created on or later than the given timestamp
diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md
index 3a3946a..cb3d1b3 100644
--- a/docs/content/development/extensions-contrib/azure.md
+++ b/docs/content/development/extensions-contrib/azure.md
@@ -33,6 +33,9 @@ The storage account is shared with the one used for Azure deep storage functiona
 
 As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz
 
+This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
+Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
+
 Sample spec:
 
 ```json
diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md
index 954179d..ad1c612 100644
--- a/docs/content/development/extensions-contrib/cloudfiles.md
+++ b/docs/content/development/extensions-contrib/cloudfiles.md
@@ -32,6 +32,9 @@ The storage account is shared with the one used for Racksapce's Cloud Files deep
 
 As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz
 
+This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
+Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
+
 Sample spec:
 
 ```json
diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md
index 28e8a1f..5f6c23e 100644
--- a/docs/content/development/extensions-contrib/google.md
+++ b/docs/content/development/extensions-contrib/google.md
@@ -27,6 +27,9 @@ This firehose ingests events, similar to the StaticS3Firehose, but from an Googl
 
 As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz
 
+This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
+Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
+
 Sample spec:
 
 ```json
diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md
index 21b95d6..5f8a07f 100644
--- a/docs/content/development/extensions-core/s3.md
+++ b/docs/content/development/extensions-core/s3.md
@@ -43,6 +43,8 @@ You can enable [server-side encryption](https://docs.aws.amazon.com/AmazonS3/lat
 ## StaticS3Firehose
 
 This firehose ingests events from a predefined list of S3 objects.
+This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
+Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
 
 Sample spec:
 
diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md
index 91553f8..b127631 100644
--- a/docs/content/ingestion/firehose.md
+++ b/docs/content/ingestion/firehose.md
@@ -20,6 +20,8 @@ For additional firehoses, please see our [extensions list](../development/extens
 
 This Firehose can be used to read the data from files on local disk.
 It can be used for POCs to ingest data on disk.
+This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#parallel-index-task).
+Since each split represents a file in this firehose, each worker task of `index_parallel` will read a file.
 A sample local firehose spec is shown below:
 
 ```json
@@ -39,6 +41,8 @@ A sample local firehose spec is shown below:
 #### HttpFirehose
 
 This Firehose can be used to read the data from remote sites via HTTP.
+This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#parallel-index-task).
+Since each split represents a file in this firehose, each worker task of `index_parallel` will read a file.
 A sample http firehose spec is shown below:
 
 ```json
diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md
new file mode 100644
index 0000000..0e9353c
--- /dev/null
+++ b/docs/content/ingestion/native_tasks.md
@@ -0,0 +1,530 @@
+---
+layout: doc_page
+---
+# Native Index Tasks
+
+Druid currently has two types of native batch indexing tasks, `index_parallel` which runs tasks
+in parallel on multiple middle manager nodes, and `index` which will run a single indexing task locally on a single
+middle manager.
+
+Parallel Index Task
+--------------------------------
+
+The Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and
+doesn't depend on other external systems like Hadoop. This task currently works in a single phase without shuffling intermediate
+data. `index_parallel` task is a supervisor task which basically generates multiple worker tasks and submits
+them to overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all
+input, they report the generated segment list to the supervisor task. The supervisor task periodically checks the worker
+task statuses. If one of them fails, it retries the failed task until the retrying number reaches the configured limit.
+If all worker tasks succeed, then it collects the reported list of generated segments and publishes those segments at once.
+
+To use this task, the `firehose` in `ioConfig` should be _splittable_. If it's not, this task runs sequentially. The
+current splittable fireshoses are [`LocalFirehose`](./firehose.html#localfirehose), [`HttpFirehose`](./firehose.html#httpfirehose)
+, [`StaticS3Firehose`](../development/extensions-core/s3.html#statics3firehose), [`StaticAzureBlobStoreFirehose`](../development/extensions-contrib/azure.html#staticazureblobstorefirehose)
+, [`StaticGoogleBlobStoreFirehose`](../development/extensions-contrib/google.html#staticgoogleblobstorefirehose), and [`StaticCloudFilesFirehose`](../development/extensions-contrib/cloudfiles.html#staticcloudfilesfirehose).
+
+The splittable firehose is responsible for generating _splits_. The supervisor task generates _worker task specs_ each of
+which specifies a split and submits worker tasks using those specs. As a result, the number of worker tasks depends on
+the implementation of splittable firehoses. Please note that multiple tasks can be created for the same worker task spec
+if one of them fails.
+
+Since this task doesn't shuffle intermediate data, it isn't available for [perfect rollup](../design/index.html). 
+
+An example ingestion spec is:
+
+```json
+{
+  "type": "index_parallel",
+  "spec": {
+    "dataSchema": {
+      "dataSource": "wikipedia_parallel_index_test",
+      "metricsSpec": [
+        {
+          "type": "count",
+              "name": "count"
+            },
+            {
+              "type": "doubleSum",
+              "name": "added",
+              "fieldName": "added"
+            },
+            {
+              "type": "doubleSum",
+              "name": "deleted",
+              "fieldName": "deleted"
+            },
+            {
+              "type": "doubleSum",
+              "name": "delta",
+              "fieldName": "delta"
+            }
+        ],
+        "granularitySpec": {
+          "segmentGranularity": "DAY",
+          "queryGranularity": "second",
+          "intervals" : [ "2013-08-31/2013-09-02" ]
+        },
+        "parser": {
+          "parseSpec": {
+            "format" : "json",
+            "timestampSpec": {
+              "column": "timestamp"
+            },
+            "dimensionsSpec": {
+              "dimensions": [
+                "page",
+                "language",
+                "user",
+                "unpatrolled",
+                "newPage",
+                "robot",
+                "anonymous",
+                "namespace",
+                "continent",
+                "country",
+                "region",
+                "city"
+              ]
+            }
+          }
+        }
+    },
+    "ioConfig": {
+        "type": "index_parallel",
+        "firehose": {
+          "type": "local",
+          "baseDir": "examples/indexing/",
+          "filter": "wikipedia_index_data*"
+        }
+    }
+  }
+}
+```
+
+#### Task Properties
+
+|property|description|required?|
+|--------|-----------|---------|
+|type|The task type, this should always be `index_parallel`.|yes|
+|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no|
+|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes|
+|context|Context containing various task configuration parameters. See below for more details.|no|
+
+#### DataSchema
+
+This field is required.
+
+See [Ingestion](../ingestion/index.html)
+
+#### IOConfig
+
+|property|description|default|required?|
+|--------|-----------|-------|---------|
+|type|The task type, this should always be `index_parallel`.|none|yes|
+|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes|
+|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no|
+
+#### TuningConfig
+
+The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details.
+
+|property|description|default|required?|
+|--------|-----------|-------|---------|
+|type|The task type, this should always be `index_parallel`.|none|yes|
+|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
+|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
+|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
+|maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|150000|no|
+|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
+|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
+|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
+|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
+|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
+|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no|
+|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
+|maxNumSubTasks|Maximum number of tasks which can be run at the same time.|Integer.MAX_VALUE|no|
+|maxRetry|Maximum number of retries on task failures.|3|no|
+|taskStatusCheckPeriodMs|Polling period in milleseconds to check running task statuses.|1000|no|
+|chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no|
+|chatHandlerNumRetries|Retries for reporting the pushed segments in worker tasks.|5|no|
+
+#### HTTP Endpoints
+
+The supervisor task provides some HTTP endpoints to get running status.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/mode`
+
+Returns 'parallel' if the indexing task is running in parallel. Otherwise, it returns 'sequential'.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/progress`
+
+Returns the current progress if the supervisor task is running in the parallel mode.
+
+An example of the result is
+
+```json
+{
+  "running":10,
+  "succeeded":0,
+  "failed":0,
+  "complete":0,
+  "total":10,
+  "expectedSucceeded":10
+}
+```
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtasks/running`
+
+Returns the task IDs of running worker tasks, or an empty list if the supervisor task is running in the sequential mode.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs`
+
+Returns all worker task specs, or an empty list if the supervisor task is running in the sequential mode.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/running`
+
+Returns running worker task specs, or an empty list if the supervisor task is running in the sequential mode.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/complete`
+
+Returns complete worker task specs, or an empty list if the supervisor task is running in the sequential mode.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}`
+
+Returns the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/state`
+
+Returns the state of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
+The returned result contains the worker task spec, a current task status if exists, and task attempt history.
+
+An example of the result is
+
+```json
+{
+  "spec": {
+    "id": "index_parallel_lineitem_2018-04-20T22:12:43.610Z_2",
+    "groupId": "index_parallel_lineitem_2018-04-20T22:12:43.610Z",
+    "supervisorTaskId": "index_parallel_lineitem_2018-04-20T22:12:43.610Z",
+    "context": null,
+    "inputSplit": {
+      "split": "/path/to/data/lineitem.tbl.5"
+    },
+    "ingestionSpec": {
+      "dataSchema": {
+        "dataSource": "lineitem",
+        "parser": {
+          "type": "hadoopyString",
+          "parseSpec": {
+            "format": "tsv",
+            "delimiter": "|",
+            "timestampSpec": {
+              "column": "l_shipdate",
+              "format": "yyyy-MM-dd"
+            },
+            "dimensionsSpec": {
+              "dimensions": [
+                "l_orderkey",
+                "l_partkey",
+                "l_suppkey",
+                "l_linenumber",
+                "l_returnflag",
+                "l_linestatus",
+                "l_shipdate",
+                "l_commitdate",
+                "l_receiptdate",
+                "l_shipinstruct",
+                "l_shipmode",
+                "l_comment"
+              ]
+            },
+            "columns": [
+              "l_orderkey",
+              "l_partkey",
+              "l_suppkey",
+              "l_linenumber",
+              "l_quantity",
+              "l_extendedprice",
+              "l_discount",
+              "l_tax",
+              "l_returnflag",
+              "l_linestatus",
+              "l_shipdate",
+              "l_commitdate",
+              "l_receiptdate",
+              "l_shipinstruct",
+              "l_shipmode",
+              "l_comment"
+            ]
+          }
+        },
+        "metricsSpec": [
+          {
+            "type": "count",
+            "name": "count"
+          },
+          {
+            "type": "longSum",
+            "name": "l_quantity",
+            "fieldName": "l_quantity",
+            "expression": null
+          },
+          {
+            "type": "doubleSum",
+            "name": "l_extendedprice",
+            "fieldName": "l_extendedprice",
+            "expression": null
+          },
+          {
+            "type": "doubleSum",
+            "name": "l_discount",
+            "fieldName": "l_discount",
+            "expression": null
+          },
+          {
+            "type": "doubleSum",
+            "name": "l_tax",
+            "fieldName": "l_tax",
+            "expression": null
+          }
+        ],
+        "granularitySpec": {
+          "type": "uniform",
+          "segmentGranularity": "YEAR",
+          "queryGranularity": {
+            "type": "none"
+          },
+          "rollup": true,
+          "intervals": [
+            "1980-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z"
+          ]
+        },
+        "transformSpec": {
+          "filter": null,
+          "transforms": []
+        }
+      },
+      "ioConfig": {
+        "type": "index_parallel",
+        "firehose": {
+          "type": "local",
+          "baseDir": "/path/to/data/",
+          "filter": "lineitem.tbl.5",
+          "parser": null
+        },
+        "appendToExisting": false
+      },
+      "tuningConfig": {
+        "type": "index_parallel",
+        "targetPartitionSize": 5000000,
+        "maxRowsInMemory": 1000000,
+        "maxTotalRows": 20000000,
+        "numShards": null,
+        "indexSpec": {
+          "bitmap": {
+            "type": "concise"
+          },
+          "dimensionCompression": "lz4",
+          "metricCompression": "lz4",
+          "longEncoding": "longs"
+        },
+        "maxPendingPersists": 0,
+        "forceExtendableShardSpecs": false,
+        "reportParseExceptions": false,
+        "pushTimeout": 0,
+        "segmentWriteOutMediumFactory": null,
+        "maxNumSubTasks": 2147483647,
+        "maxRetry": 3,
+        "taskStatusCheckPeriodMs": 1000,
+        "chatHandlerTimeout": "PT10S",
+        "chatHandlerNumRetries": 5,
+        "logParseExceptions": false,
+        "maxParseExceptions": 2147483647,
+        "maxSavedParseExceptions": 0,
+        "forceGuaranteedRollup": false,
+        "buildV9Directly": true
+      }
+    }
+  },
+  "currentStatus": {
+    "id": "index_sub_lineitem_2018-04-20T22:16:29.922Z",
+    "type": "index_sub",
+    "createdTime": "2018-04-20T22:16:29.925Z",
+    "queueInsertionTime": "2018-04-20T22:16:29.929Z",
+    "statusCode": "RUNNING",
+    "duration": -1,
+    "location": {
+      "host": null,
+      "port": -1,
+      "tlsPort": -1
+    },
+    "dataSource": "lineitem",
+    "errorMsg": null
+  },
+  "taskHistory": []
+}
+```
+
+* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/history`
+
+Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
+
+Local Index Task
+----------------
+
+The Local Index Task is designed to be used for smaller data sets. The task executes within the indexing service. The grammar of the index task is as follows:
+
+```json
+{
+  "type" : "index",
+  "spec" : {
+    "dataSchema" : {
+      "dataSource" : "wikipedia",
+      "parser" : {
+        "type" : "string",
+        "parseSpec" : {
+          "format" : "json",
+          "timestampSpec" : {
+            "column" : "timestamp",
+            "format" : "auto"
+          },
+          "dimensionsSpec" : {
+            "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"],
+            "dimensionExclusions" : [],
+            "spatialDimensions" : []
+          }
+        }
+      },
+      "metricsSpec" : [
+        {
+          "type" : "count",
+          "name" : "count"
+        },
+        {
+          "type" : "doubleSum",
+          "name" : "added",
+          "fieldName" : "added"
+        },
+        {
+          "type" : "doubleSum",
+          "name" : "deleted",
+          "fieldName" : "deleted"
+        },
+        {
+          "type" : "doubleSum",
+          "name" : "delta",
+          "fieldName" : "delta"
+        }
+      ],
+      "granularitySpec" : {
+        "type" : "uniform",
+        "segmentGranularity" : "DAY",
+        "queryGranularity" : "NONE",
+        "intervals" : [ "2013-08-31/2013-09-01" ]
+      }
+    },
+    "ioConfig" : {
+      "type" : "index",
+      "firehose" : {
+        "type" : "local",
+        "baseDir" : "examples/indexing/",
+        "filter" : "wikipedia_data.json"
+       }
+    },
+    "tuningConfig" : {
+      "type" : "index",
+      "targetPartitionSize" : 5000000,
+      "maxRowsInMemory" : 1000000
+    }
+  }
+}
+```
+
+#### Task Properties
+
+|property|description|required?|
+|--------|-----------|---------|
+|type|The task type, this should always be "index".|yes|
+|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no|
+|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes|
+|context|Context containing various task configuration parameters. See below for more details.|no|
+
+#### DataSchema
+
+This field is required.
+
+See [Ingestion](../ingestion/index.html)
+
+#### IOConfig
+
+|property|description|default|required?|
+|--------|-----------|-------|---------|
+|type|The task type, this should always be "index".|none|yes|
+|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes|
+|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no|
+
+#### TuningConfig
+
+The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details.
+
+|property|description|default|required?|
+|--------|-----------|-------|---------|
+|type|The task type, this should always be "index".|none|yes|
+|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
+|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
+|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
+|maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|20000000|no|
+|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
+|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
+|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
+|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
+|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment pushing modes__ section.|false|no|
+|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
+|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no|
+|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
+
+#### IndexSpec
+
+The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column
+compression formats. The indexSpec is optional and default parameters will be used if not specified.
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
+|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
+|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)|
+|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
+
+##### Bitmap types
+
+For Concise bitmaps:
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|type|String|Must be `concise`.|yes|
+
+For Roaring bitmaps:
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|type|String|Must be `roaring`.|yes|
+|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
+
+#### Segment pushing modes
+
+While ingesting data using the Index task, it creates segments from the input data and pushes them. For segment pushing,
+the Index task supports two segment pushing modes, i.e., _bulk pushing mode_ and _incremental pushing mode_ for
+[perfect rollup and best-effort rollup](./design/index.html), respectively.
+
+In the bulk pushing mode, every segment is pushed at the very end of the index task. Until then, created segments
+are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a
+problem due to limited storage capacity, and is not recommended to use in production.
+
+On the contrary, in the incremental pushing mode, segments are incrementally pushed, that is they can be pushed
+in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory
+and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds,
+the index task immediately pushes all segments created until that moment, cleans all pushed segments up, and
+continues to ingest remaining data.
+
+To enable bulk pushing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot
+be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig.
diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md
index dace297..680bd43 100644
--- a/docs/content/ingestion/tasks.md
+++ b/docs/content/ingestion/tasks.md
@@ -13,178 +13,10 @@ Segment Creation Tasks
 
 See [batch ingestion](../ingestion/batch-ingestion.html).
 
-### Index Task
+### Native Index Tasks
 
-The Index Task is a simpler variation of the Index Hadoop task that is designed to be used for smaller data sets. The task executes within the indexing service and does not require an external Hadoop setup to use. The grammar of the index task is as follows:
-
-```json
-{
-  "type" : "index",
-  "spec" : {
-    "dataSchema" : {
-      "dataSource" : "wikipedia",
-      "parser" : {
-        "type" : "string",
-        "parseSpec" : {
-          "format" : "json",
-          "timestampSpec" : {
-            "column" : "timestamp",
-            "format" : "auto"
-          },
-          "dimensionsSpec" : {
-            "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"],
-            "dimensionExclusions" : [],
-            "spatialDimensions" : []
-          }
-        }
-      },
-      "metricsSpec" : [
-        {
-          "type" : "count",
-          "name" : "count"
-        },
-        {
-          "type" : "doubleSum",
-          "name" : "added",
-          "fieldName" : "added"
-        },
-        {
-          "type" : "doubleSum",
-          "name" : "deleted",
-          "fieldName" : "deleted"
-        },
-        {
-          "type" : "doubleSum",
-          "name" : "delta",
-          "fieldName" : "delta"
-        }
-      ],
-      "granularitySpec" : {
-        "type" : "uniform",
-        "segmentGranularity" : "DAY",
-        "queryGranularity" : "NONE",
-        "intervals" : [ "2013-08-31/2013-09-01" ]
-      }
-    },
-    "ioConfig" : {
-      "type" : "index",
-      "firehose" : {
-        "type" : "local",
-        "baseDir" : "examples/indexing/",
-        "filter" : "wikipedia_data.json"
-       }
-    },
-    "tuningConfig" : {
-      "type" : "index",
-      "targetPartitionSize" : 5000000,
-      "maxRowsInMemory" : 1000000
-    }
-  }
-}
-```
-
-#### Task Properties
-
-|property|description|required?|
-|--------|-----------|---------|
-|type|The task type, this should always be "index".|yes|
-|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no|
-|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes|
-|context|Context containing various task configuration parameters. See below for more details.|no|
-
-#### Task Priority
-
-Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking)
-
-Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted.
-
-Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority.
-
-|task type|default priority|
-|---------|----------------|
-|Realtime index task|75|
-|Batch index task|50|
-|Merge/Append/Compaction task|25|
-|Other tasks|0|
-
-You can override the task priority by setting your priority in the task context like below.
-
-```json
-"context" : {
-  "priority" : 100
-}
-```
-
-#### DataSchema
-
-This field is required.
-
-See [Ingestion](../ingestion/index.html)
-
-#### IOConfig
-
-|property|description|default|required?|
-|--------|-----------|-------|---------|
-|type|The task type, this should always be "index".|none|yes|
-|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes|
-|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no|
-
-#### TuningConfig
-
-The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details.
-
-|property|description|default|required?|
-|--------|-----------|-------|---------|
-|type|The task type, this should always be "index".|none|yes|
-|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
-|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
-|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
-|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|20000000|no|
-|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
-|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
-|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
-|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
-|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no|
-|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
-|publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no|
-|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
-
-#### IndexSpec
-
-The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column
-compression formats. The indexSpec is optional and default parameters will be used if not specified.
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
-|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
-|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)|
-|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
-
-##### Bitmap types
-
-For Concise bitmaps:
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|type|String|Must be `concise`.|yes|
-
-For Roaring bitmaps:
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|type|String|Must be `roaring`.|yes|
-|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
-
-#### Segment publishing modes
-
-While ingesting data using the Index task, it creates segments from the input data and publishes them. For segment publishing, the Index task supports two segment publishing modes, i.e., _bulk publishing mode_ and _incremental publishing mode_ for [perfect rollup and best-effort rollup](./design/index.html), respectively.
-
-In the bulk publishing mode, every segment is published at the very end of the index task. Until then, created segments are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a problem due to limited storage capacity, and is not recommended to use in production.
-
-On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to i [...]
-
-To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig.
+Druid provides a native index task which doesn't need any dependencies on other systems.
+See [native index tasks](./native_tasks.html) for more details.
 
 Segment Merging Tasks
 ---------------------
@@ -382,6 +214,33 @@ The values are described below.
 
 Unlike the hadoop convert task, the indexing service task draws its output path from the indexing service's configuration.
 
+#### IndexSpec
+
+The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column
+compression formats. The indexSpec is optional and default parameters will be used if not specified.
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
+|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
+|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)|
+|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
+
+##### Bitmap types
+
+For Concise bitmaps:
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|type|String|Must be `concise`.|yes|
+
+For Roaring bitmaps:
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|type|String|Must be `roaring`.|yes|
+|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
+
 ### Noop Task
 
 These tasks start, sleep for a time and are used only for testing. The available grammar is:
@@ -428,3 +287,27 @@ A task writing data into a dataSource must acquire exclusive locks for target in
 Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release 
 locks early if they desire. Task ids are unique by naming them using UUIDs or the timestamp in which the task was created. 
 Tasks are also part of a "task group", which is a set of tasks that can share interval locks.
+
+Task Priority
+-------------
+
+Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking)
+
+Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted.
+
+Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority.
+
+|task type|default priority|
+|---------|----------------|
+|Realtime index task|75|
+|Batch index task|50|
+|Merge/Append/Compaction task|25|
+|Other tasks|0|
+
+You can override the task priority by setting your priority in the task context like below.
+
+```json
+"context" : {
+  "priority" : 100
+}
+```
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java
index ffeaf30..d29b15c 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java
@@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.storage.azure.AzureByteSource;
@@ -33,6 +36,7 @@ import io.druid.storage.azure.AzureUtils;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
@@ -143,4 +147,18 @@ public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFi
   {
     return AzureUtils.AZURE_RETRY;
   }
+
+  @Override
+  public FiniteFirehoseFactory<StringInputRowParser, AzureBlob> withSplit(InputSplit<AzureBlob> split)
+  {
+    return new StaticAzureBlobStoreFirehoseFactory(
+        azureStorage,
+        Collections.singletonList(split.get()),
+        getMaxCacheCapacityBytes(),
+        getMaxFetchCapacityBytes(),
+        getPrefetchTriggerBytes(),
+        getFetchTimeout(),
+        getMaxFetchRetry()
+    );
+  }
 }
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
index 49786e1..1a31e4b 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
@@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Predicate;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.java.util.common.logger.Logger;
@@ -34,6 +37,7 @@ import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
@@ -142,4 +146,18 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho
   {
     return CloudFilesUtils.CLOUDFILESRETRY;
   }
+
+  @Override
+  public FiniteFirehoseFactory<StringInputRowParser, CloudFilesBlob> withSplit(InputSplit<CloudFilesBlob> split)
+  {
+    return new StaticCloudFilesFirehoseFactory(
+        cloudFilesApi,
+        Collections.singletonList(split.get()),
+        getMaxCacheCapacityBytes(),
+        getMaxFetchCapacityBytes(),
+        getPrefetchTriggerBytes(),
+        getFetchTimeout(),
+        getMaxFetchRetry()
+    );
+  }
 }
diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java
index 48db831..a49ee8c 100644
--- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java
+++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java
@@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Predicate;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.storage.google.GoogleByteSource;
@@ -32,6 +35,7 @@ import io.druid.storage.google.GoogleUtils;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
@@ -134,5 +138,19 @@ public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesF
   {
     return GoogleUtils::isRetryable;
   }
+
+  @Override
+  public FiniteFirehoseFactory<StringInputRowParser, GoogleBlob> withSplit(InputSplit<GoogleBlob> split)
+  {
+    return new StaticGoogleBlobStoreFirehoseFactory(
+        storage,
+        Collections.singletonList(split.get()),
+        getMaxCacheCapacityBytes(),
+        getMaxFetchCapacityBytes(),
+        getPrefetchTriggerBytes(),
+        getFetchTimeout(),
+        getMaxFetchRetry()
+    );
+  }
 }
 
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
index 2f8f224..a9d5c98 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
@@ -33,6 +33,7 @@ import io.druid.indexer.TaskStatus;
 import io.druid.indexing.appenderator.ActionBasedSegmentAllocator;
 import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
 import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.SegmentAllocateAction;
 import io.druid.indexing.common.actions.TaskActionClient;
 import io.druid.indexing.common.stats.RowIngestionMetersFactory;
 import io.druid.indexing.common.task.AbstractTask;
@@ -258,7 +259,19 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
   {
     return new StreamAppenderatorDriver(
         appenderator,
-        new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema),
+        new ActionBasedSegmentAllocator(
+            toolbox.getTaskActionClient(),
+            dataSchema,
+            (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
+                schema.getDataSource(),
+                row.getTimestamp(),
+                schema.getGranularitySpec().getQueryGranularity(),
+                schema.getGranularitySpec().getSegmentGranularity(),
+                sequenceName,
+                previousSegmentId,
+                skipSegmentLineageCheck
+            )
+        ),
         toolbox.getSegmentHandoffNotifierFactory(),
         new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
         toolbox.getDataSegmentKiller(),
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
index 5484b19..7e65d7f 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
@@ -21,80 +21,31 @@ package io.druid.indexing.kafka;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import io.druid.indexer.TaskLocation;
-import io.druid.indexer.TaskStatus;
-import io.druid.indexing.common.RetryPolicy;
-import io.druid.indexing.common.RetryPolicyConfig;
-import io.druid.indexing.common.RetryPolicyFactory;
+import io.druid.indexing.common.IndexTaskClient;
 import io.druid.indexing.common.TaskInfoProvider;
-import io.druid.java.util.common.IAE;
-import io.druid.java.util.common.IOE;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
-import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.jackson.JacksonUtils;
 import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.java.util.http.client.HttpClient;
-import io.druid.java.util.http.client.Request;
-import io.druid.java.util.http.client.response.FullResponseHandler;
 import io.druid.java.util.http.client.response.FullResponseHolder;
-import io.druid.segment.realtime.firehose.ChatHandlerResource;
-import org.jboss.netty.channel.ChannelException;
 import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
-import org.joda.time.Period;
 
-import javax.ws.rs.core.MediaType;
+import javax.annotation.Nullable;
 import java.io.IOException;
-import java.net.Socket;
-import java.net.URI;
-import java.nio.charset.StandardCharsets;
 import java.util.Collections;
 import java.util.Map;
 import java.util.TreeMap;
-import java.util.concurrent.Callable;
 
-public class KafkaIndexTaskClient
+public class KafkaIndexTaskClient extends IndexTaskClient
 {
-  public static class NoTaskLocationException extends RuntimeException
-  {
-    public NoTaskLocationException(String message)
-    {
-      super(message);
-    }
-  }
-
-  public static class TaskNotRunnableException extends RuntimeException
-  {
-    public TaskNotRunnableException(String message)
-    {
-      super(message);
-    }
-  }
-
-  public static final int MAX_RETRY_WAIT_SECONDS = 10;
-
-  private static final int MIN_RETRY_WAIT_SECONDS = 2;
   private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class);
-  private static final String BASE_PATH = "/druid/worker/v1/chat";
-  private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5;
-  private static final TreeMap EMPTY_TREE_MAP = new TreeMap();
-
-  private final HttpClient httpClient;
-  private final ObjectMapper jsonMapper;
-  private final TaskInfoProvider taskInfoProvider;
-  private final Duration httpTimeout;
-  private final RetryPolicyFactory retryPolicyFactory;
-  private final ListeningExecutorService executorService;
-  private final long numRetries;
+  private static final TreeMap<Integer, Map<Integer, Long>> EMPTY_TREE_MAP = new TreeMap<>();
 
   public KafkaIndexTaskClient(
       HttpClient httpClient,
@@ -106,27 +57,7 @@ public class KafkaIndexTaskClient
       long numRetries
   )
   {
-    this.httpClient = httpClient;
-    this.jsonMapper = jsonMapper;
-    this.taskInfoProvider = taskInfoProvider;
-    this.httpTimeout = httpTimeout;
-    this.numRetries = numRetries;
-    this.retryPolicyFactory = createRetryPolicyFactory();
-
-    this.executorService = MoreExecutors.listeningDecorator(
-        Execs.multiThreaded(
-            numThreads,
-            StringUtils.format(
-                "KafkaIndexTaskClient-%s-%%d",
-                dataSource
-            )
-        )
-    );
-  }
-
-  public void close()
-  {
-    executorService.shutdownNow();
+    super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries);
   }
 
   public boolean stop(final String id, final boolean publish)
@@ -134,10 +65,10 @@ public class KafkaIndexTaskClient
     log.debug("Stop task[%s] publish[%s]", id, publish);
 
     try {
-      final FullResponseHolder response = submitRequest(
+      final FullResponseHolder response = submitRequestWithEmptyContent(
           id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true
       );
-      return response.getStatus().getCode() / 100 == 2;
+      return isSuccess(response);
     }
     catch (NoTaskLocationException e) {
       return false;
@@ -157,10 +88,11 @@ public class KafkaIndexTaskClient
     log.debug("Resume task[%s]", id);
 
     try {
-      final FullResponseHolder response = submitRequest(id, HttpMethod.POST, "resume", null, true);
-      return response.getStatus().getCode() / 100 == 2;
+      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true);
+      return isSuccess(response);
     }
-    catch (NoTaskLocationException e) {
+    catch (NoTaskLocationException | IOException e) {
+      log.warn(e, "Exception while stopping task [%s]", id);
       return false;
     }
   }
@@ -170,7 +102,7 @@ public class KafkaIndexTaskClient
     log.debug("Pause task[%s]", id);
 
     try {
-      final FullResponseHolder response = submitRequest(
+      final FullResponseHolder response = submitRequestWithEmptyContent(
           id,
           HttpMethod.POST,
           "pause",
@@ -180,18 +112,17 @@ public class KafkaIndexTaskClient
 
       if (response.getStatus().equals(HttpResponseStatus.OK)) {
         log.info("Task [%s] paused successfully", id);
-        return jsonMapper.readValue(response.getContent(), new TypeReference<Map<Integer, Long>>()
+        return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
         {
         });
       }
 
-      final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
       while (true) {
         if (getStatus(id) == KafkaIndexTask.Status.PAUSED) {
           return getCurrentOffsets(id, true);
         }
 
-        final Duration delay = retryPolicy.getAndIncrementRetryDelay();
+        final Duration delay = newRetryPolicy().getAndIncrementRetryDelay();
         if (delay == null) {
           log.error("Task [%s] failed to pause, aborting", id);
           throw new ISE("Task [%s] failed to pause, aborting", id);
@@ -223,8 +154,8 @@ public class KafkaIndexTaskClient
     log.debug("GetStatus task[%s]", id);
 
     try {
-      final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "status", null, true);
-      return jsonMapper.readValue(response.getContent(), KafkaIndexTask.Status.class);
+      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true);
+      return deserialize(response.getContent(), KafkaIndexTask.Status.class);
     }
     catch (NoTaskLocationException e) {
       return KafkaIndexTask.Status.NOT_STARTED;
@@ -234,15 +165,16 @@ public class KafkaIndexTaskClient
     }
   }
 
+  @Nullable
   public DateTime getStartTime(final String id)
   {
     log.debug("GetStartTime task[%s]", id);
 
     try {
-      final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "time/start", null, true);
+      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true);
       return response.getContent() == null || response.getContent().isEmpty()
              ? null
-             : jsonMapper.readValue(response.getContent(), DateTime.class);
+             : deserialize(response.getContent(), DateTime.class);
     }
     catch (NoTaskLocationException e) {
       return null;
@@ -257,7 +189,7 @@ public class KafkaIndexTaskClient
     log.debug("GetMovingAverages task[%s]", id);
 
     try {
-      final FullResponseHolder response = submitRequest(
+      final FullResponseHolder response = submitRequestWithEmptyContent(
           id,
           HttpMethod.GET,
           "rowStats",
@@ -266,7 +198,7 @@ public class KafkaIndexTaskClient
       );
       return response.getContent() == null || response.getContent().isEmpty()
              ? Collections.emptyMap()
-             : jsonMapper.readValue(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
+             : deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
     }
     catch (NoTaskLocationException e) {
       return Collections.emptyMap();
@@ -276,27 +208,19 @@ public class KafkaIndexTaskClient
     }
   }
 
-  public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
-  {
-    return executorService.submit(
-        new Callable<Map<String, Object>>()
-        {
-          @Override
-          public Map<String, Object> call()
-          {
-            return getMovingAverages(id);
-          }
-        }
-    );
-  }
-
   public Map<Integer, Long> getCurrentOffsets(final String id, final boolean retry)
   {
     log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry);
 
     try {
-      final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/current", null, retry);
-      return jsonMapper.readValue(response.getContent(), new TypeReference<Map<Integer, Long>>()
+      final FullResponseHolder response = submitRequestWithEmptyContent(
+          id,
+          HttpMethod.GET,
+          "offsets/current",
+          null,
+          retry
+      );
+      return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
       {
       });
     }
@@ -312,10 +236,13 @@ public class KafkaIndexTaskClient
   {
     log.debug("GetCheckpoints task[%s] retry[%s]", id, retry);
     try {
-      final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "checkpoints", null, retry);
-      return jsonMapper.readValue(response.getContent(), new TypeReference<TreeMap<Integer, TreeMap<Integer, Long>>>()
-      {
-      });
+      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry);
+      return deserialize(
+          response.getContent(),
+          new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
+          {
+          }
+      );
     }
     catch (NoTaskLocationException e) {
       return EMPTY_TREE_MAP;
@@ -330,9 +257,7 @@ public class KafkaIndexTaskClient
       final boolean retry
   )
   {
-    return executorService.submit(
-        () -> getCheckpoints(id, retry)
-    );
+    return doAsync(() -> getCheckpoints(id, retry));
   }
 
   public Map<Integer, Long> getEndOffsets(final String id)
@@ -340,8 +265,8 @@ public class KafkaIndexTaskClient
     log.debug("GetEndOffsets task[%s]", id);
 
     try {
-      final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/end", null, true);
-      return jsonMapper.readValue(response.getContent(), new TypeReference<Map<Integer, Long>>()
+      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true);
+      return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
       {
       });
     }
@@ -357,322 +282,72 @@ public class KafkaIndexTaskClient
       final String id,
       final Map<Integer, Long> endOffsets,
       final boolean finalize
-  )
+  ) throws IOException
   {
     log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize);
 
     try {
-      final FullResponseHolder response = submitRequest(
+      final FullResponseHolder response = submitJsonRequest(
           id,
           HttpMethod.POST,
           "offsets/end",
           StringUtils.format("finish=%s", finalize),
-          jsonMapper.writeValueAsBytes(endOffsets),
+          serialize(endOffsets),
           true
       );
-      return response.getStatus().getCode() / 100 == 2;
+      return isSuccess(response);
     }
     catch (NoTaskLocationException e) {
       return false;
     }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
   }
 
   public ListenableFuture<Boolean> stopAsync(final String id, final boolean publish)
   {
-    return executorService.submit(
-        new Callable<Boolean>()
-        {
-          @Override
-          public Boolean call()
-          {
-            return stop(id, publish);
-          }
-        }
-    );
+    return doAsync(() -> stop(id, publish));
   }
 
   public ListenableFuture<Boolean> resumeAsync(final String id)
   {
-    return executorService.submit(
-        new Callable<Boolean>()
-        {
-          @Override
-          public Boolean call()
-          {
-            return resume(id);
-          }
-        }
-    );
+    return doAsync(() -> resume(id));
   }
 
   public ListenableFuture<Map<Integer, Long>> pauseAsync(final String id)
   {
-    return executorService.submit(
-        new Callable<Map<Integer, Long>>()
-        {
-          @Override
-          public Map<Integer, Long> call()
-          {
-            return pause(id);
-          }
-        }
-    );
+    return doAsync(() -> pause(id));
   }
 
   public ListenableFuture<KafkaIndexTask.Status> getStatusAsync(final String id)
   {
-    return executorService.submit(
-        new Callable<KafkaIndexTask.Status>()
-        {
-          @Override
-          public KafkaIndexTask.Status call()
-          {
-            return getStatus(id);
-          }
-        }
-    );
+    return doAsync(() -> getStatus(id));
   }
 
   public ListenableFuture<DateTime> getStartTimeAsync(final String id)
   {
-    return executorService.submit(
-        new Callable<DateTime>()
-        {
-          @Override
-          public DateTime call()
-          {
-            return getStartTime(id);
-          }
-        }
-    );
+    return doAsync(() -> getStartTime(id));
   }
 
   public ListenableFuture<Map<Integer, Long>> getCurrentOffsetsAsync(final String id, final boolean retry)
   {
-    return executorService.submit(
-        new Callable<Map<Integer, Long>>()
-        {
-          @Override
-          public Map<Integer, Long> call()
-          {
-            return getCurrentOffsets(id, retry);
-          }
-        }
-    );
+    return doAsync(() -> getCurrentOffsets(id, retry));
   }
 
   public ListenableFuture<Map<Integer, Long>> getEndOffsetsAsync(final String id)
   {
-    return executorService.submit(
-        new Callable<Map<Integer, Long>>()
-        {
-          @Override
-          public Map<Integer, Long> call()
-          {
-            return getEndOffsets(id);
-          }
-        }
-    );
+    return doAsync(() -> getEndOffsets(id));
   }
 
   public ListenableFuture<Boolean> setEndOffsetsAsync(
-      final String id, final Map<Integer, Long> endOffsets, final boolean finalize
+      final String id,
+      final Map<Integer, Long> endOffsets,
+      final boolean finalize
   )
   {
-    return executorService.submit(
-        new Callable<Boolean>()
-        {
-          @Override
-          public Boolean call()
-          {
-            return setEndOffsets(id, endOffsets, finalize);
-          }
-        }
-    );
-  }
-
-  @VisibleForTesting
-  RetryPolicyFactory createRetryPolicyFactory()
-  {
-    // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary
-    // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in
-    // the middle of persisting to disk and doesn't respond immediately.
-    return new RetryPolicyFactory(
-        new RetryPolicyConfig()
-            .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS))
-            .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS))
-            .setMaxRetryCount(numRetries)
-    );
-  }
-
-  @VisibleForTesting
-  void checkConnection(String host, int port) throws IOException
-  {
-    new Socket(host, port).close();
-  }
-
-  private FullResponseHolder submitRequest(String id, HttpMethod method, String pathSuffix, String query, boolean retry)
-  {
-    return submitRequest(id, method, pathSuffix, query, new byte[0], retry);
+    return doAsync(() -> setEndOffsets(id, endOffsets, finalize));
   }
 
-  private FullResponseHolder submitRequest(
-      String id,
-      HttpMethod method,
-      String pathSuffix,
-      String query,
-      byte[] content,
-      boolean retry
-  )
+  public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
   {
-    final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
-    while (true) {
-      FullResponseHolder response = null;
-      Request request = null;
-      TaskLocation location = TaskLocation.unknown();
-      String path = StringUtils.format("%s/%s/%s", BASE_PATH, id, pathSuffix);
-
-      Optional<TaskStatus> status = taskInfoProvider.getTaskStatus(id);
-      if (!status.isPresent() || !status.get().isRunnable()) {
-        throw new TaskNotRunnableException(StringUtils.format(
-            "Aborting request because task [%s] is not runnable",
-            id
-        ));
-      }
-
-      String host = location.getHost();
-      String scheme = "";
-      int port = -1;
-
-      try {
-        location = taskInfoProvider.getTaskLocation(id);
-        if (location.equals(TaskLocation.unknown())) {
-          throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", id));
-        }
-
-        host = location.getHost();
-        scheme = location.getTlsPort() >= 0 ? "https" : "http";
-        port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort();
-
-        // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently
-        // for tasks that happen to still be starting up, so test the connection first to keep the logs clean.
-        checkConnection(host, port);
-
-        try {
-          URI serviceUri = new URI(
-              scheme,
-              null,
-              host,
-              port,
-              path,
-              query,
-              null
-          );
-          request = new Request(method, serviceUri.toURL());
-
-          // used to validate that we are talking to the correct worker
-          request.addHeader(ChatHandlerResource.TASK_ID_HEADER, id);
-
-          if (content.length > 0) {
-            request.setContent(MediaType.APPLICATION_JSON, content);
-          }
-
-          log.debug("HTTP %s: %s", method.getName(), serviceUri.toString());
-          response = httpClient.go(request, new FullResponseHandler(StandardCharsets.UTF_8), httpTimeout).get();
-
-        }
-        catch (IOException | ChannelException ioce) {
-          throw ioce;
-        }
-        catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException(ie);
-        }
-        catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-
-        int responseCode = response.getStatus().getCode();
-        if (responseCode / 100 == 2) {
-          return response;
-        } else if (responseCode == 400) { // don't bother retrying if it's a bad request
-          throw new IAE("Received 400 Bad Request with body: %s", response.getContent());
-        } else {
-          throw new IOE("Received status [%d]", responseCode);
-        }
-      }
-      catch (IOException | ChannelException e) {
-
-        // Since workers are free to move tasks around to different ports, there is a chance that a task may have been
-        // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header
-        // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the
-        // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then
-        // we will wait for a short period then retry the request indefinitely, expecting the task's location to
-        // eventually be updated.
-
-        final Duration delay;
-        if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) {
-          String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER);
-          if (headerId != null && !headerId.equals(id)) {
-            log.warn(
-                "Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s",
-                id, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS
-            );
-            delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS);
-          } else {
-            delay = retryPolicy.getAndIncrementRetryDelay();
-          }
-        } else {
-          delay = retryPolicy.getAndIncrementRetryDelay();
-        }
-        String urlForLog = (request != null
-                            ? request.getUrl().toString()
-                            : StringUtils.format(
-                                "%s://%s:%d%s",
-                                scheme,
-                                host,
-                                port,
-                                path
-                            ));
-        if (!retry) {
-          // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was
-          // for informational purposes only) so don't log a scary stack trace
-          log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage());
-          throw new RuntimeException(e);
-        } else if (delay == null) {
-          log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog);
-          throw new RuntimeException(e);
-        } else {
-          try {
-            final long sleepTime = delay.getMillis();
-            log.debug(
-                "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])",
-                (response != null ? response.getStatus().getCode() : "no response"),
-                urlForLog,
-                new Duration(sleepTime).toString(),
-                (response != null ? response.getContent() : e.getMessage())
-            );
-            Thread.sleep(sleepTime);
-          }
-          catch (InterruptedException e2) {
-            Thread.currentThread().interrupt();
-            e.addSuppressed(e2);
-            throw new RuntimeException(e);
-          }
-        }
-      }
-      catch (NoTaskLocationException e) {
-        log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or "
-                 + "may have already completed", id);
-        throw e;
-      }
-      catch (Exception e) {
-        log.warn(e, "Exception while sending request");
-        throw e;
-      }
-    }
+    return doAsync(() -> getMovingAverages(id));
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
index 0d88a15..72f8b13 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
@@ -21,13 +21,14 @@ package io.druid.indexing.kafka;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.inject.Inject;
-import io.druid.java.util.http.client.HttpClient;
 import io.druid.guice.annotations.EscalatedGlobal;
 import io.druid.guice.annotations.Json;
 import io.druid.indexing.common.TaskInfoProvider;
+import io.druid.indexing.common.task.IndexTaskClientFactory;
+import io.druid.java.util.http.client.HttpClient;
 import org.joda.time.Duration;
 
-public class KafkaIndexTaskClientFactory
+public class KafkaIndexTaskClientFactory implements IndexTaskClientFactory<KafkaIndexTaskClient>
 {
   private HttpClient httpClient;
   private ObjectMapper mapper;
@@ -42,6 +43,7 @@ public class KafkaIndexTaskClientFactory
     this.mapper = mapper;
   }
 
+  @Override
   public KafkaIndexTaskClient build(
       TaskInfoProvider taskInfoProvider,
       String dataSource,
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java
index 6f00ed2..3771d92 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java
@@ -24,6 +24,9 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
 import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Binder;
+import com.google.inject.TypeLiteral;
+import io.druid.guice.LazySingleton;
+import io.druid.indexing.common.task.IndexTaskClientFactory;
 import io.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
 import io.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
 import io.druid.initialization.DruidModule;
@@ -50,5 +53,10 @@ public class KafkaIndexTaskModule implements DruidModule
   @Override
   public void configure(Binder binder)
   {
+    binder.bind(
+        new TypeLiteral<IndexTaskClientFactory<KafkaIndexTaskClient>>()
+        {
+        }
+    ).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class);
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
index 57e82ca..bfa0d89 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
@@ -29,6 +29,7 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexer.TaskStatus;
+import io.druid.indexing.common.IndexTaskClient;
 import io.druid.indexing.common.TaskInfoProvider;
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.java.util.common.DateTimes;
@@ -56,6 +57,7 @@ import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.io.IOException;
 import java.net.URL;
 import java.util.Collections;
 import java.util.List;
@@ -134,7 +136,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
   }
 
   @Test
-  public void testNoTaskLocation()
+  public void testNoTaskLocation() throws IOException
   {
     reset(taskInfoProvider);
     expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
@@ -158,7 +160,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
   @Test
   public void testTaskNotRunnableException()
   {
-    expectedException.expect(KafkaIndexTaskClient.TaskNotRunnableException.class);
+    expectedException.expect(IndexTaskClient.TaskNotRunnableException.class);
     expectedException.expectMessage("Aborting request because task [test-id] is not runnable");
 
     reset(taskInfoProvider);
@@ -950,7 +952,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
     }
 
     @Override
-    void checkConnection(String host, int port)
+    protected void checkConnection(String host, int port)
     {
     }
   }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
index c3553b2..1e30a95 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -50,6 +50,7 @@ import io.druid.discovery.DruidNodeAnnouncer;
 import io.druid.discovery.LookupNodeService;
 import io.druid.indexer.TaskState;
 import io.druid.indexer.TaskStatus;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
 import io.druid.indexing.common.SegmentLoaderFactory;
 import io.druid.indexing.common.TaskLock;
@@ -2050,6 +2051,7 @@ public class KafkaIndexTaskTest
     taskLockbox = new TaskLockbox(taskStorage);
     final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
         taskLockbox,
+        taskStorage,
         metadataStorageCoordinator,
         emitter,
         new SupervisorManager(null)
@@ -2073,7 +2075,8 @@ public class KafkaIndexTaskTest
             );
             return true;
           }
-        }
+        },
+        new Counters()
     );
     final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
         taskStorage,
diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java
index b1266a6..00fa357 100644
--- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java
+++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java
@@ -30,11 +30,15 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Lists;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.java.util.common.IAE;
 import io.druid.java.util.common.IOE;
 import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.storage.s3.ServerSideEncryptingAmazonS3;
 import io.druid.storage.s3.S3Utils;
@@ -44,6 +48,7 @@ import java.io.InputStream;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
@@ -52,7 +57,7 @@ import java.util.stream.Collectors;
 /**
  * Builds firehoses that read from a predefined list of S3 objects and then dry up.
  */
-public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory<S3ObjectSummary>
+public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
 {
   private static final Logger log = new Logger(StaticS3FirehoseFactory.class);
   private static final int MAX_LISTING_LENGTH = 1024;
@@ -108,20 +113,12 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
   }
 
   @Override
-  protected Collection<S3ObjectSummary> initObjects() throws IOException
+  protected Collection<URI> initObjects() throws IOException
   {
     // Here, the returned s3 objects contain minimal information without data.
     // Getting data is deferred until openObjectStream() is called for each object.
     if (!uris.isEmpty()) {
-      return uris.stream()
-                 .map(
-                     uri -> {
-                       final String s3Bucket = uri.getAuthority();
-                       final String key = S3Utils.extractS3Key(uri);
-                       return S3Utils.getSingleObjectSummary(s3Client, s3Bucket, key);
-                     }
-                 )
-                 .collect(Collectors.toList());
+      return uris;
     } else {
       final List<S3ObjectSummary> objects = new ArrayList<>();
       for (URI uri : prefixes) {
@@ -167,18 +164,21 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
           }
         }
       }
-      return objects;
+      return objects.stream().map(StaticS3FirehoseFactory::toUri).collect(Collectors.toList());
     }
   }
 
   @Override
-  protected InputStream openObjectStream(S3ObjectSummary object) throws IOException
+  protected InputStream openObjectStream(URI object) throws IOException
   {
     try {
       // Get data of the given object and open an input stream
-      final S3Object s3Object = s3Client.getObject(object.getBucketName(), object.getKey());
+      final String bucket = object.getAuthority();
+      final String key = S3Utils.extractS3Key(object);
+
+      final S3Object s3Object = s3Client.getObject(bucket, key);
       if (s3Object == null) {
-        throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", object.getBucketName(), object.getKey());
+        throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", bucket, key);
       }
       return s3Object.getObjectContent();
     }
@@ -188,17 +188,20 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
   }
 
   @Override
-  protected InputStream openObjectStream(S3ObjectSummary object, long start) throws IOException
+  protected InputStream openObjectStream(URI object, long start) throws IOException
   {
-    final GetObjectRequest request = new GetObjectRequest(object.getBucketName(), object.getKey());
+    final String bucket = object.getAuthority();
+    final String key = S3Utils.extractS3Key(object);
+
+    final GetObjectRequest request = new GetObjectRequest(bucket, key);
     request.setRange(start);
     try {
       final S3Object s3Object = s3Client.getObject(request);
       if (s3Object == null) {
         throw new ISE(
             "Failed to get an s3 object for bucket[%s], key[%s], and start[%d]",
-            object.getBucketName(),
-            object.getKey(),
+            bucket,
+            key,
             start
         );
       }
@@ -210,9 +213,9 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
   }
 
   @Override
-  protected InputStream wrapObjectStream(S3ObjectSummary object, InputStream stream) throws IOException
+  protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
   {
-    return CompressionUtils.decompress(stream, object.getKey());
+    return CompressionUtils.decompress(stream, S3Utils.extractS3Key(object));
   }
 
   @Override
@@ -255,4 +258,40 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
   {
     return S3Utils.S3RETRY;
   }
+
+  @Override
+  public FiniteFirehoseFactory<StringInputRowParser, URI> withSplit(InputSplit<URI> split)
+  {
+    return new StaticS3FirehoseFactory(
+        s3Client,
+        Collections.singletonList(split.get()),
+        null,
+        getMaxCacheCapacityBytes(),
+        getMaxFetchCapacityBytes(),
+        getPrefetchTriggerBytes(),
+        getFetchTimeout(),
+        getMaxFetchRetry()
+    );
+  }
+
+  /**
+   * Create an {@link URI} from the given {@link S3ObjectSummary}. The result URI is composed as below.
+   *
+   * <pre>
+   * {@code s3://{BUCKET_NAME}/{OBJECT_KEY}}
+   * </pre>
+   */
+  private static URI toUri(S3ObjectSummary object)
+  {
+    final String originalAuthority = object.getBucketName();
+    final String originalPath = object.getKey();
+    final String authority = originalAuthority.endsWith("/") ?
+                             originalAuthority.substring(0, originalAuthority.length() - 1) :
+                             originalAuthority;
+    final String path = originalPath.startsWith("/") ?
+                        originalPath.substring(1, originalPath.length()) :
+                        originalPath;
+
+    return URI.create(StringUtils.format("s3://%s/%s", authority, path));
+  }
 }
diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java
index 42c2942..c3097ad 100644
--- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java
+++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java
@@ -21,6 +21,9 @@ package io.druid.firehose.s3;
 
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.Module;
@@ -33,22 +36,30 @@ import com.google.inject.Binder;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Provides;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.initialization.DruidModule;
+import io.druid.storage.s3.S3Utils;
+import org.easymock.EasyMock;
 import io.druid.storage.s3.ServerSideEncryptingAmazonS3;
 import io.druid.storage.s3.NoopServerSideEncryption;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
  */
 public class StaticS3FirehoseFactoryTest
 {
+  private static final AmazonS3Client S3_ClIENT = EasyMock.createNiceMock(AmazonS3Client.class);
   private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
-      new AmazonS3Client(),
+      S3_ClIENT,
       new NoopServerSideEncryption()
   );
 
@@ -81,6 +92,60 @@ public class StaticS3FirehoseFactoryTest
     Assert.assertEquals(factory, outputFact);
   }
 
+  @Test
+  public void testWithSplit() throws IOException
+  {
+    final List<URI> uris = Arrays.asList(
+        URI.create("s3://foo/bar/file.gz"),
+        URI.create("s3://bar/foo/file2.gz")
+    );
+    uris.sort(Comparator.comparing(URI::toString));
+
+    uris.forEach(StaticS3FirehoseFactoryTest::addExpectedObjject);
+    EasyMock.replay(S3_ClIENT);
+
+    final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory(
+        SERVICE,
+        uris,
+        null,
+        2048L,
+        1024L,
+        512L,
+        100L,
+        5
+    );
+    final List<FiniteFirehoseFactory<StringInputRowParser, URI>> subFactories = factory
+        .getSplits()
+        .map(factory::withSplit)
+        .sorted(Comparator.comparing(eachFactory -> {
+          final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) eachFactory;
+          return staticS3FirehoseFactory.getUris().toString();
+        }))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals(uris.size(), subFactories.size());
+    for (int i = 0; i < uris.size(); i++) {
+      final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) subFactories.get(i);
+      final List<URI> subFactoryUris = staticS3FirehoseFactory.getUris();
+      Assert.assertEquals(1, subFactoryUris.size());
+      Assert.assertEquals(uris.get(i), subFactoryUris.get(0));
+    }
+  }
+
+  private static void addExpectedObjject(URI uri)
+  {
+    final String s3Bucket = uri.getAuthority();
+    final String key = S3Utils.extractS3Key(uri);
+    final S3ObjectSummary objectSummary = new S3ObjectSummary();
+    objectSummary.setBucketName(s3Bucket);
+    objectSummary.setKey(key);
+    final ListObjectsV2Result result = new ListObjectsV2Result();
+    result.setBucketName(s3Bucket);
+    result.setKeyCount(1);
+    result.getObjectSummaries().add(objectSummary);
+    EasyMock.expect(SERVICE.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))).andReturn(result);
+  }
+
   private static ObjectMapper createObjectMapper(DruidModule baseModule)
   {
     final Injector injector = Guice.createInjector(
diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java
index 38a2731..53cd699 100644
--- a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java
+++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java
@@ -20,7 +20,6 @@
 package io.druid.indexing.appenderator;
 
 import io.druid.data.input.InputRow;
-import io.druid.indexing.common.actions.SegmentAllocateAction;
 import io.druid.indexing.common.actions.TaskActionClient;
 import io.druid.segment.indexing.DataSchema;
 import io.druid.segment.realtime.appenderator.SegmentAllocator;
@@ -32,14 +31,17 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
 {
   private final TaskActionClient taskActionClient;
   private final DataSchema dataSchema;
+  private final SegmentAllocateActionGenerator actionGenerator;
 
   public ActionBasedSegmentAllocator(
       TaskActionClient taskActionClient,
-      DataSchema dataSchema
+      DataSchema dataSchema,
+      SegmentAllocateActionGenerator actionGenerator
   )
   {
     this.taskActionClient = taskActionClient;
     this.dataSchema = dataSchema;
+    this.actionGenerator = actionGenerator;
   }
 
   @Override
@@ -51,15 +53,7 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
   ) throws IOException
   {
     return taskActionClient.submit(
-        new SegmentAllocateAction(
-            dataSchema.getDataSource(),
-            row.getTimestamp(),
-            dataSchema.getGranularitySpec().getQueryGranularity(),
-            dataSchema.getGranularitySpec().getSegmentGranularity(),
-            sequenceName,
-            previousSegmentId,
-            skipSegmentLineageCheck
-        )
+        actionGenerator.generate(dataSchema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck)
     );
   }
 }
diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java
new file mode 100644
index 0000000..4a0352f
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.appenderator;
+
+import io.druid.data.input.InputRow;
+import io.druid.indexing.common.actions.TaskAction;
+import io.druid.segment.indexing.DataSchema;
+import io.druid.segment.realtime.appenderator.SegmentIdentifier;
+
+/**
+ * This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a
+ * {@link TaskAction<SegmentIdentifier>} which is submitted to overlords to allocate a new segment.
+ * The {@link #generate} method can return any implementation of {@link TaskAction<SegmentIdentifier>}.
+ *
+ * @see io.druid.indexing.common.actions.SegmentAllocateAction
+ * @see io.druid.indexing.common.actions.SurrogateAction
+ */
+public interface SegmentAllocateActionGenerator
+{
+  TaskAction<SegmentIdentifier> generate(
+      DataSchema dataSchema,
+      InputRow row,
+      String sequenceName,
+      String previousSegmentId,
+      boolean skipSegmentLineageCheck
+  );
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/Counters.java b/indexing-service/src/main/java/io/druid/indexing/common/Counters.java
new file mode 100644
index 0000000..6563c7d
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/Counters.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common;
+
+import com.google.common.util.concurrent.AtomicDouble;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BinaryOperator;
+
+public class Counters
+{
+  private final ConcurrentMap<String, AtomicInteger> intCounters = new ConcurrentHashMap<>();
+  private final ConcurrentMap<String, AtomicDouble> doubleCounters = new ConcurrentHashMap<>();
+  private final ConcurrentMap<String, AtomicReference> objectCounters = new ConcurrentHashMap<>();
+
+  public int increment(String key, int val)
+  {
+    return intCounters.computeIfAbsent(key, k -> new AtomicInteger()).addAndGet(val);
+  }
+
+  public double increment(String key, double val)
+  {
+    return doubleCounters.computeIfAbsent(key, k -> new AtomicDouble()).addAndGet(val);
+  }
+
+  public Object increment(String key, Object obj, BinaryOperator mergeFunction)
+  {
+    return objectCounters.computeIfAbsent(key, k -> new AtomicReference()).accumulateAndGet(obj, mergeFunction);
+  }
+
+  @Nullable
+  public Integer getIntCounter(String key)
+  {
+    final AtomicInteger atomicInteger = intCounters.get(key);
+    return atomicInteger == null ? null : atomicInteger.get();
+  }
+
+  @Nullable
+  public Double getDoubleCounter(String key)
+  {
+    final AtomicDouble atomicDouble = doubleCounters.get(key);
+    return atomicDouble == null ? null : atomicDouble.get();
+  }
+
+  @Nullable
+  public Object getObjectCounter(String key)
+  {
+    final AtomicReference atomicReference = objectCounters.get(key);
+    return atomicReference == null ? null : atomicReference.get();
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java
new file mode 100644
index 0000000..631d4b4
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java
@@ -0,0 +1,381 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import io.druid.indexer.TaskLocation;
+import io.druid.indexer.TaskStatus;
+import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.IOE;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.FullResponseHandler;
+import io.druid.java.util.http.client.response.FullResponseHolder;
+import io.druid.segment.realtime.firehose.ChatHandlerResource;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.joda.time.Duration;
+import org.joda.time.Period;
+
+import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.net.Socket;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.Callable;
+
+/**
+ * Abstract class to communicate with index tasks via HTTP. This class provides interfaces to serialize/deserialize
+ * data and send an HTTP request.
+ */
+public abstract class IndexTaskClient implements AutoCloseable
+{
+  public static class NoTaskLocationException extends RuntimeException
+  {
+    public NoTaskLocationException(String message)
+    {
+      super(message);
+    }
+  }
+
+  public static class TaskNotRunnableException extends RuntimeException
+  {
+    public TaskNotRunnableException(String message)
+    {
+      super(message);
+    }
+  }
+
+  public static final int MAX_RETRY_WAIT_SECONDS = 10;
+
+  private static final EmittingLogger log = new EmittingLogger(IndexTaskClient.class);
+  private static final String BASE_PATH = "/druid/worker/v1/chat";
+  private static final int MIN_RETRY_WAIT_SECONDS = 2;
+  private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5;
+
+  private final HttpClient httpClient;
+  private final ObjectMapper objectMapper;
+  private final TaskInfoProvider taskInfoProvider;
+  private final Duration httpTimeout;
+  private final RetryPolicyFactory retryPolicyFactory;
+  private final ListeningExecutorService executorService;
+
+  public IndexTaskClient(
+      HttpClient httpClient,
+      ObjectMapper objectMapper,
+      TaskInfoProvider taskInfoProvider,
+      Duration httpTimeout,
+      String callerId,
+      int numThreads,
+      long numRetries
+  )
+  {
+    this.httpClient = httpClient;
+    this.objectMapper = objectMapper;
+    this.taskInfoProvider = taskInfoProvider;
+    this.httpTimeout = httpTimeout;
+    this.retryPolicyFactory = initializeRetryPolicyFactory(numRetries);
+    this.executorService = MoreExecutors.listeningDecorator(
+        Execs.multiThreaded(
+            numThreads,
+            StringUtils.format(
+                "IndexTaskClient-%s-%%d",
+                callerId
+            )
+        )
+    );
+  }
+
+  private static RetryPolicyFactory initializeRetryPolicyFactory(long numRetries)
+  {
+    // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary
+    // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in
+    // the middle of persisting to disk and doesn't respond immediately.
+    return new RetryPolicyFactory(
+        new RetryPolicyConfig()
+            .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS))
+            .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS))
+            .setMaxRetryCount(numRetries)
+    );
+  }
+
+  protected HttpClient getHttpClient()
+  {
+    return httpClient;
+  }
+
+  protected RetryPolicy newRetryPolicy()
+  {
+    return retryPolicyFactory.makeRetryPolicy();
+  }
+
+  protected <T> T deserialize(String content, TypeReference<T> typeReference) throws IOException
+  {
+    return objectMapper.readValue(content, typeReference);
+  }
+
+  protected <T> T deserialize(String content, Class<T> typeReference) throws IOException
+  {
+    return objectMapper.readValue(content, typeReference);
+  }
+
+  protected byte[] serialize(Object value) throws JsonProcessingException
+  {
+    return objectMapper.writeValueAsBytes(value);
+  }
+
+  protected <T> ListenableFuture<T> doAsync(Callable<T> callable)
+  {
+    return executorService.submit(callable);
+  }
+
+  protected boolean isSuccess(FullResponseHolder responseHolder)
+  {
+    return responseHolder.getStatus().getCode() / 100 == 2;
+  }
+
+  @VisibleForTesting
+  protected void checkConnection(String host, int port) throws IOException
+  {
+    new Socket(host, port).close();
+  }
+
+  protected FullResponseHolder submitRequestWithEmptyContent(
+      String taskId,
+      HttpMethod method,
+      String pathSuffix,
+      @Nullable String query,
+      boolean retry
+  ) throws IOException, ChannelException, NoTaskLocationException
+  {
+    return submitRequest(taskId, null, method, pathSuffix, query, new byte[0], retry);
+  }
+
+  /**
+   * To use this method, {@link #objectMapper} should be a jsonMapper.
+   */
+  protected FullResponseHolder submitJsonRequest(
+      String taskId,
+      HttpMethod method,
+      String pathSuffix,
+      @Nullable String query,
+      byte[] content,
+      boolean retry
+  ) throws IOException, ChannelException, NoTaskLocationException
+  {
+    return submitRequest(taskId, MediaType.APPLICATION_JSON, method, pathSuffix, query, content, retry);
+  }
+
+  /**
+   * To use this method, {@link #objectMapper} should be a smileMapper.
+   */
+  protected FullResponseHolder submitSmileRequest(
+      String taskId,
+      HttpMethod method,
+      String pathSuffix,
+      @Nullable String query,
+      byte[] content,
+      boolean retry
+  ) throws IOException, ChannelException, NoTaskLocationException
+  {
+    return submitRequest(taskId, SmileMediaTypes.APPLICATION_JACKSON_SMILE, method, pathSuffix, query, content, retry);
+  }
+
+  /**
+   * Sends an HTTP request to the task of the specified {@code taskId} and returns a response if it succeeded.
+   */
+  private FullResponseHolder submitRequest(
+      String taskId,
+      @Nullable String mediaType, // nullable if content is empty
+      HttpMethod method,
+      String pathSuffix,
+      @Nullable String query,
+      byte[] content,
+      boolean retry
+  ) throws IOException, ChannelException, NoTaskLocationException
+  {
+    final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
+
+    while (true) {
+      FullResponseHolder response = null;
+      Request request = null;
+      TaskLocation location = TaskLocation.unknown();
+      String path = StringUtils.format("%s/%s/%s", BASE_PATH, taskId, pathSuffix);
+
+      Optional<TaskStatus> status = taskInfoProvider.getTaskStatus(taskId);
+      if (!status.isPresent() || !status.get().isRunnable()) {
+        throw new TaskNotRunnableException(StringUtils.format(
+            "Aborting request because task [%s] is not runnable",
+            taskId
+        ));
+      }
+
+      String host = location.getHost();
+      String scheme = "";
+      int port = -1;
+
+      try {
+        location = taskInfoProvider.getTaskLocation(taskId);
+        if (location.equals(TaskLocation.unknown())) {
+          throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", taskId));
+        }
+
+        host = location.getHost();
+        scheme = location.getTlsPort() >= 0 ? "https" : "http";
+        port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort();
+
+        // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently
+        // for tasks that happen to still be starting up, so test the connection first to keep the logs clean.
+        checkConnection(host, port);
+
+        try {
+          URI serviceUri = new URI(
+              scheme,
+              null,
+              host,
+              port,
+              path,
+              query,
+              null
+          );
+          request = new Request(method, serviceUri.toURL());
+
+          // used to validate that we are talking to the correct worker
+          request.addHeader(ChatHandlerResource.TASK_ID_HEADER, taskId);
+
+          if (content.length > 0) {
+            request.setContent(Preconditions.checkNotNull(mediaType, "mediaType"), content);
+          }
+
+          log.debug("HTTP %s: %s", method.getName(), serviceUri.toString());
+          response = httpClient.go(request, new FullResponseHandler(StandardCharsets.UTF_8), httpTimeout).get();
+        }
+        catch (IOException | ChannelException ioce) {
+          throw ioce;
+        }
+        catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          throw new RuntimeException(ie);
+        }
+        catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+
+        int responseCode = response.getStatus().getCode();
+        if (responseCode / 100 == 2) {
+          return response;
+        } else if (responseCode == 400) { // don't bother retrying if it's a bad request
+          throw new IAE("Received 400 Bad Request with body: %s", response.getContent());
+        } else {
+          throw new IOE("Received status [%d]", responseCode);
+        }
+      }
+      catch (IOException | ChannelException e) {
+
+        // Since workers are free to move tasks around to different ports, there is a chance that a task may have been
+        // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header
+        // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the
+        // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then
+        // we will wait for a short period then retry the request indefinitely, expecting the task's location to
+        // eventually be updated.
+
+        final Duration delay;
+        if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) {
+          String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER);
+          if (headerId != null && !headerId.equals(taskId)) {
+            log.warn(
+                "Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s",
+                taskId, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS
+            );
+            delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS);
+          } else {
+            delay = retryPolicy.getAndIncrementRetryDelay();
+          }
+        } else {
+          delay = retryPolicy.getAndIncrementRetryDelay();
+        }
+        String urlForLog = (request != null
+                            ? request.getUrl().toString()
+                            : StringUtils.nonStrictFormat(
+                                "%s://%s:%d%s",
+                                scheme,
+                                host,
+                                port,
+                                path
+                            ));
+        if (!retry) {
+          // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was
+          // for informational purposes only) so don't log a scary stack trace
+          log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage());
+          throw e;
+        } else if (delay == null) {
+          log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog);
+          throw e;
+        } else {
+          try {
+            final long sleepTime = delay.getMillis();
+            log.debug(
+                "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])",
+                (response != null ? response.getStatus().getCode() : "no response"),
+                urlForLog,
+                new Duration(sleepTime).toString(),
+                (response != null ? response.getContent() : e.getMessage())
+            );
+            Thread.sleep(sleepTime);
+          }
+          catch (InterruptedException e2) {
+            Thread.currentThread().interrupt();
+            e.addSuppressed(e2);
+            throw new RuntimeException(e);
+          }
+        }
+      }
+      catch (NoTaskLocationException e) {
+        log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or "
+                 + "may have already completed", taskId);
+        throw e;
+      }
+      catch (Exception e) {
+        log.warn(e, "Exception while sending request");
+        throw e;
+      }
+    }
+  }
+
+  @Override
+  public void close()
+  {
+    executorService.shutdownNow();
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java
new file mode 100644
index 0000000..8befeb0
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.actions;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Optional;
+import io.druid.indexing.common.task.Task;
+import io.druid.java.util.common.ISE;
+
+/**
+ * Perform the given action using {@link #surrogateId} on behalf of the caller task.
+ */
+public class SurrogateAction<ReturnType, ActionType extends TaskAction<ReturnType>> implements TaskAction<ReturnType>
+{
+  private final String surrogateId;
+  private final ActionType taskAction;
+
+  @JsonCreator
+  public SurrogateAction(
+      @JsonProperty("surrogateId") String surrogateId,
+      @JsonProperty("taskAction") ActionType taskAction
+  )
+  {
+    this.surrogateId = surrogateId;
+    this.taskAction = taskAction;
+  }
+
+  @JsonProperty
+  public String getSurrogateId()
+  {
+    return surrogateId;
+  }
+
+  @JsonProperty
+  public ActionType getTaskAction()
+  {
+    return taskAction;
+  }
+
+  @Override
+  public TypeReference<ReturnType> getReturnTypeReference()
+  {
+    return taskAction.getReturnTypeReference();
+  }
+
+  @Override
+  public ReturnType perform(Task task, TaskActionToolbox toolbox)
+  {
+    final Optional<Task> maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId);
+    if (maybeSurrogateTask.isPresent()) {
+      return taskAction.perform(maybeSurrogateTask.get(), toolbox);
+    } else {
+      throw new ISE("Can't find surrogate task[%s]", surrogateId);
+    }
+  }
+
+  @Override
+  public boolean isAudited()
+  {
+    return taskAction.isAudited();
+  }
+
+  @Override
+  public String toString()
+  {
+    return "SurrogateAction{" +
+           "surrogateId='" + surrogateId + '\'' +
+           ", taskAction=" + taskAction +
+           '}';
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java
index 8246429..fe1ae79 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java
@@ -38,11 +38,15 @@ import io.druid.indexing.common.task.Task;
     @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class),
     @JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class),
     @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class),
-    @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class)
+    @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class),
+    @JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class)
 })
 public interface TaskAction<RetType>
 {
   TypeReference<RetType> getReturnTypeReference(); // T_T
   RetType perform(Task task, TaskActionToolbox toolbox);
   boolean isAudited();
+
+  @Override
+  String toString();
 }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java
index 38667fa..586a979 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java
@@ -42,7 +42,12 @@ public class TaskActionPreconditions
   )
   {
     if (!isLockCoversSegments(task, taskLockbox, segments)) {
-      throw new ISE("Segments not covered by locks for task: %s", task.getId());
+      throw new ISE(
+          "Segments[%s] are not covered by locks[%s] for task[%s]",
+          segments,
+          taskLockbox.findLocksForTask(task),
+          task.getId()
+      );
     }
   }
 
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java
index c439905..a547a9d 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java
@@ -20,30 +20,38 @@
 package io.druid.indexing.common.actions;
 
 import com.google.inject.Inject;
-import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
 import io.druid.indexing.overlord.TaskLockbox;
+import io.druid.indexing.overlord.TaskStorage;
 import io.druid.indexing.overlord.supervisor.SupervisorManager;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 
 public class TaskActionToolbox
 {
   private final TaskLockbox taskLockbox;
+  private final TaskStorage taskStorage;
   private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
   private final ServiceEmitter emitter;
   private final SupervisorManager supervisorManager;
+  private final Counters counters;
 
   @Inject
   public TaskActionToolbox(
       TaskLockbox taskLockbox,
+      TaskStorage taskStorage,
       IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
       ServiceEmitter emitter,
-      SupervisorManager supervisorManager
+      SupervisorManager supervisorManager,
+      Counters counters
   )
   {
     this.taskLockbox = taskLockbox;
+    this.taskStorage = taskStorage;
     this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
     this.emitter = emitter;
     this.supervisorManager = supervisorManager;
+    this.counters = counters;
   }
 
   public TaskLockbox getTaskLockbox()
@@ -51,6 +59,11 @@ public class TaskActionToolbox
     return taskLockbox;
   }
 
+  public TaskStorage getTaskStorage()
+  {
+    return taskStorage;
+  }
+
   public IndexerMetadataStorageCoordinator getIndexerMetadataStorageCoordinator()
   {
     return indexerMetadataStorageCoordinator;
@@ -65,4 +78,9 @@ public class TaskActionToolbox
   {
     return supervisorManager;
   }
+
+  public Counters getCounters()
+  {
+    return counters;
+  }
 }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java
index d40e42d..e5f9eb3 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java
@@ -78,7 +78,7 @@ public abstract class AbstractTask implements Task
     this.context = context == null ? new HashMap<>() : context;
   }
 
-  static String getOrMakeId(String id, final String typeName, String dataSource)
+  public static String getOrMakeId(String id, final String typeName, String dataSource)
   {
     return getOrMakeId(id, typeName, dataSource, null);
   }
@@ -227,7 +227,7 @@ public abstract class AbstractTask implements Task
     return Objects.hashCode(id, groupId, dataSource, context);
   }
 
-  static List<TaskLock> getTaskLocks(TaskActionClient client) throws IOException
+  public static List<TaskLock> getTaskLocks(TaskActionClient client) throws IOException
   {
     return client.submit(new LockListAction());
   }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
index fa93014..6c367fd 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
@@ -47,6 +47,7 @@ import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
 import io.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
 import io.druid.indexing.common.TaskReport;
 import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.SegmentAllocateAction;
 import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
 import io.druid.indexing.common.actions.TaskActionClient;
 import io.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec;
@@ -718,7 +719,19 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
   {
     return new StreamAppenderatorDriver(
         appenderator,
-        new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema),
+        new ActionBasedSegmentAllocator(
+            toolbox.getTaskActionClient(),
+            dataSchema,
+            (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
+                schema.getDataSource(),
+                row.getTimestamp(),
+                schema.getGranularitySpec().getQueryGranularity(),
+                schema.getGranularitySpec().getSegmentGranularity(),
+                sequenceName,
+                previousSegmentId,
+                skipSegmentLineageCheck
+            )
+        ),
         toolbox.getSegmentHandoffNotifierFactory(),
         new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
         toolbox.getDataSegmentKiller(),
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java
new file mode 100644
index 0000000..243a7ad
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.client.indexing.TaskStatusResponse;
+import io.druid.indexer.TaskLocation;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexing.common.TaskInfoProvider;
+
+public class ClientBasedTaskInfoProvider implements TaskInfoProvider
+{
+  private final IndexingServiceClient client;
+
+  @Inject
+  public ClientBasedTaskInfoProvider(IndexingServiceClient client)
+  {
+    this.client = client;
+  }
+
+  @Override
+  public TaskLocation getTaskLocation(String id)
+  {
+    final TaskStatusResponse response = client.getTaskStatus(id);
+    return response == null ? TaskLocation.unknown() : response.getStatus().getLocation();
+  }
+
+  @Override
+  public Optional<TaskStatus> getTaskStatus(String id)
+  {
+    final TaskStatusResponse response = client.getTaskStatus(id);
+    return response == null ?
+           Optional.absent() :
+           Optional.of(TaskStatus.fromCode(id, response.getStatus().getState()));
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java
index f91eaf9..6d4cfba 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java
@@ -50,6 +50,7 @@ import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
 import io.druid.indexing.common.TaskReport;
 import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.SegmentAllocateAction;
 import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
 import io.druid.indexing.common.actions.TaskActionClient;
 import io.druid.indexing.common.stats.RowIngestionMeters;
@@ -204,7 +205,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
     );
   }
 
-  IndexTask(
+  public IndexTask(
       String id,
       String groupId,
       TaskResource resource,
@@ -800,8 +801,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
           }
 
           determinePartitionsMeters.incrementUnparseable();
-          if (determinePartitionsMeters.getUnparseable() > ingestionSchema.getTuningConfig()
-                                                                          .getMaxParseExceptions()) {
+          if (determinePartitionsMeters.getUnparseable() > ingestionSchema.getTuningConfig().getMaxParseExceptions()) {
             throw new RuntimeException("Max parse exceptions exceeded, terminating task...");
           }
         }
@@ -854,7 +854,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
       final TaskToolbox toolbox,
       final DataSchema dataSchema,
       final ShardSpecs shardSpecs,
-      Map<Interval, String> versions,
+      final Map<Interval, String> versions,
       final FirehoseFactory firehoseFactory,
       final File firehoseTempDir
   ) throws IOException, InterruptedException
@@ -908,7 +908,19 @@ public class IndexTask extends AbstractTask implements ChatHandler
       segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> lookup.get(sequenceName);
     } else if (ioConfig.isAppendToExisting()) {
       // Append mode: Allocate segments as needed using Overlord APIs.
-      segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema);
+      segmentAllocator = new ActionBasedSegmentAllocator(
+          toolbox.getTaskActionClient(),
+          dataSchema,
+          (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
+              schema.getDataSource(),
+              row.getTimestamp(),
+              schema.getGranularitySpec().getQueryGranularity(),
+              schema.getGranularitySpec().getSegmentGranularity(),
+              sequenceName,
+              previousSegmentId,
+              skipSegmentLineageCheck
+          )
+      );
     } else {
       // Overwrite mode, non-guaranteed rollup: We can make up our own segment ids but we don't know them in advance.
       final Map<Interval, AtomicInteger> counters = new HashMap<>();
diff --git a/server/src/main/java/io/druid/segment/indexing/IOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java
similarity index 69%
copy from server/src/main/java/io/druid/segment/indexing/IOConfig.java
copy to indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java
index a278847..54b0c32 100644
--- a/server/src/main/java/io/druid/segment/indexing/IOConfig.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java
@@ -17,17 +17,13 @@
  * under the License.
  */
 
-package io.druid.segment.indexing;
+package io.druid.indexing.common.task;
 
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.druid.indexing.common.IndexTaskClient;
+import io.druid.indexing.common.TaskInfoProvider;
+import org.joda.time.Duration;
 
-/**
- */
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
-@JsonSubTypes(value = {
-    @JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class),
-})
-public interface IOConfig
+public interface IndexTaskClientFactory<T extends IndexTaskClient>
 {
+  T build(TaskInfoProvider taskInfoProvider, String callerId, int numThreads, Duration httpTimeout, long numRetries);
 }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java
index 727c35e..0c6dfcd 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java
@@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexing.common.TaskToolbox;
 import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.task.batch.parallel.ParallelIndexSubTask;
+import io.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask;
 import io.druid.query.Query;
 import io.druid.query.QueryRunner;
 
@@ -50,6 +52,8 @@ import java.util.Map;
     @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class),
     @JsonSubTypes.Type(name = "restore", value = RestoreTask.class),
     @JsonSubTypes.Type(name = "index", value = IndexTask.class),
+    @JsonSubTypes.Type(name = ParallelIndexSupervisorTask.TYPE, value = ParallelIndexSupervisorTask.class),
+    @JsonSubTypes.Type(name = ParallelIndexSubTask.TYPE, value = ParallelIndexSubTask.class),
     @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
     @JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class),
     @JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class),
diff --git a/server/src/main/java/io/druid/segment/indexing/IOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
similarity index 54%
copy from server/src/main/java/io/druid/segment/indexing/IOConfig.java
copy to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
index a278847..9af9672 100644
--- a/server/src/main/java/io/druid/segment/indexing/IOConfig.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
@@ -17,17 +17,26 @@
  * under the License.
  */
 
-package io.druid.segment.indexing;
+package io.druid.indexing.common.task.batch.parallel;
 
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import io.druid.data.input.FirehoseFactory;
+import io.druid.indexing.common.task.IndexTask.IndexIOConfig;
+
+import javax.annotation.Nullable;
 
 /**
+ * Same with {@link IndexIOConfig} except its JSON type name.
  */
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
-@JsonSubTypes(value = {
-    @JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class),
-})
-public interface IOConfig
+@JsonTypeName("index_parallel")
+public class ParallelIndexIOConfig extends IndexIOConfig
 {
+  public ParallelIndexIOConfig(
+      @JsonProperty("firehose") FirehoseFactory firehoseFactory,
+      @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting
+  )
+  {
+    super(firehoseFactory, appendToExisting);
+  }
 }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java
new file mode 100644
index 0000000..9912eb1
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.segment.indexing.DataSchema;
+import io.druid.segment.indexing.IngestionSpec;
+
+public class ParallelIndexIngestionSpec extends IngestionSpec<ParallelIndexIOConfig, ParallelIndexTuningConfig>
+{
+  private final DataSchema dataSchema;
+  private final ParallelIndexIOConfig ioConfig;
+  private final ParallelIndexTuningConfig tuningConfig;
+
+  @JsonCreator
+  public ParallelIndexIngestionSpec(
+      @JsonProperty("dataSchema") DataSchema dataSchema,
+      @JsonProperty("ioConfig") ParallelIndexIOConfig ioConfig,
+      @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig
+  )
+  {
+    super(dataSchema, ioConfig, tuningConfig);
+
+    this.dataSchema = dataSchema;
+    this.ioConfig = ioConfig;
+    this.tuningConfig = tuningConfig == null ? ParallelIndexTuningConfig.defaultConfig() : tuningConfig;
+  }
+
+  @Override
+  @JsonProperty("dataSchema")
+  public DataSchema getDataSchema()
+  {
+    return dataSchema;
+  }
+
+  @Override
+  @JsonProperty("ioConfig")
+  public ParallelIndexIOConfig getIOConfig()
+  {
+    return ioConfig;
+  }
+
+  @Override
+  @JsonProperty("tuningConfig")
+  public ParallelIndexTuningConfig getTuningConfig()
+  {
+    return tuningConfig;
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java
new file mode 100644
index 0000000..a9c7e78
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.data.input.Firehose;
+import io.druid.data.input.FirehoseFactory;
+import io.druid.data.input.InputRow;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexing.appenderator.ActionBasedSegmentAllocator;
+import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
+import io.druid.indexing.common.TaskLockType;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.LockTryAcquireAction;
+import io.druid.indexing.common.actions.SegmentAllocateAction;
+import io.druid.indexing.common.actions.SurrogateAction;
+import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.task.AbstractTask;
+import io.druid.indexing.common.task.ClientBasedTaskInfoProvider;
+import io.druid.indexing.common.task.IndexTask;
+import io.druid.indexing.common.task.IndexTaskClientFactory;
+import io.druid.indexing.common.task.TaskResource;
+import io.druid.indexing.common.task.Tasks;
+import io.druid.indexing.firehose.IngestSegmentFirehoseFactory;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.common.parsers.ParseException;
+import io.druid.query.DruidMetrics;
+import io.druid.segment.indexing.DataSchema;
+import io.druid.segment.indexing.RealtimeIOConfig;
+import io.druid.segment.indexing.granularity.GranularitySpec;
+import io.druid.segment.realtime.FireDepartment;
+import io.druid.segment.realtime.FireDepartmentMetrics;
+import io.druid.segment.realtime.RealtimeMetricsMonitor;
+import io.druid.segment.realtime.appenderator.Appenderator;
+import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
+import io.druid.segment.realtime.appenderator.Appenderators;
+import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
+import io.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
+import io.druid.segment.realtime.appenderator.SegmentAllocator;
+import io.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import io.druid.timeline.DataSegment;
+import org.apache.commons.io.FileUtils;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A worker task of {@link ParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task
+ * generates and pushes segments, and reports them to the {@link ParallelIndexSupervisorTask} instead of
+ * publishing on its own.
+ */
+public class ParallelIndexSubTask extends AbstractTask
+{
+  public static final String TYPE = "index_sub";
+
+  private static final Logger log = new Logger(ParallelIndexSubTask.class);
+
+  private final int numAttempts;
+  private final ParallelIndexIngestionSpec ingestionSchema;
+  private final String supervisorTaskId;
+  private final IndexingServiceClient indexingServiceClient;
+  private final IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory;
+
+  @JsonCreator
+  public ParallelIndexSubTask(
+      // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask
+      @JsonProperty("id") @Nullable final String id,
+      @JsonProperty("groupId") final String groupId,
+      @JsonProperty("resource") final TaskResource taskResource,
+      @JsonProperty("supervisorTaskId") final String supervisorTaskId,
+      @JsonProperty("numAttempts") final int numAttempts, // zero-based counting
+      @JsonProperty("spec") final ParallelIndexIngestionSpec ingestionSchema,
+      @JsonProperty("context") final Map<String, Object> context,
+      @JacksonInject IndexingServiceClient indexingServiceClient,
+      @JacksonInject IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory
+  )
+  {
+    super(
+        getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()),
+        groupId,
+        taskResource,
+        ingestionSchema.getDataSchema().getDataSource(),
+        context
+    );
+
+    if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) {
+      throw new UnsupportedOperationException("Guaranteed rollup is not supported");
+    }
+
+    this.numAttempts = numAttempts;
+    this.ingestionSchema = ingestionSchema;
+    this.supervisorTaskId = supervisorTaskId;
+    this.indexingServiceClient = indexingServiceClient;
+    this.taskClientFactory = taskClientFactory;
+  }
+
+  @Override
+  public int getPriority()
+  {
+    return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
+  }
+
+  @Override
+  public String getType()
+  {
+    return TYPE;
+  }
+
+  @Override
+  public boolean isReady(TaskActionClient taskActionClient)
+  {
+    final Optional<SortedSet<Interval>> intervals = ingestionSchema.getDataSchema()
+                                                                   .getGranularitySpec()
+                                                                   .bucketIntervals();
+
+    return !intervals.isPresent() || checkLockAcquired(taskActionClient, intervals.get());
+  }
+
+  private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet<Interval> intervals)
+  {
+    try {
+      tryAcquireExclusiveSurrogateLocks(actionClient, intervals);
+      return true;
+    }
+    catch (Exception e) {
+      log.error(e, "Failed to acquire locks for intervals[%s]", intervals);
+      return false;
+    }
+  }
+
+  @JsonProperty
+  public int getNumAttempts()
+  {
+    return numAttempts;
+  }
+
+  @JsonProperty("spec")
+  public ParallelIndexIngestionSpec getIngestionSchema()
+  {
+    return ingestionSchema;
+  }
+
+  @JsonProperty
+  public String getSupervisorTaskId()
+  {
+    return supervisorTaskId;
+  }
+
+  @Override
+  public TaskStatus run(final TaskToolbox toolbox) throws Exception
+  {
+    final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
+
+    if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
+      // pass toolbox to Firehose
+      ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox);
+    }
+
+    final File firehoseTempDir = toolbox.getFirehoseTemporaryDir();
+    // Firehose temporary directory is automatically removed when this IndexTask completes.
+    FileUtils.forceMkdir(firehoseTempDir);
+
+    final ParallelIndexTaskClient taskClient = taskClientFactory.build(
+        new ClientBasedTaskInfoProvider(indexingServiceClient),
+        getId(),
+        1, // always use a single http thread
+        ingestionSchema.getTuningConfig().getChatHandlerTimeout(),
+        ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
+    );
+    final List<DataSegment> pushedSegments = generateAndPushSegments(
+        toolbox,
+        taskClient,
+        firehoseFactory,
+        firehoseTempDir
+    );
+    taskClient.report(supervisorTaskId, pushedSegments);
+
+    return TaskStatus.success(getId());
+  }
+
+  private void tryAcquireExclusiveSurrogateLocks(
+      TaskActionClient client,
+      SortedSet<Interval> intervals
+  )
+      throws IOException
+  {
+    for (Interval interval : Tasks.computeCompactIntervals(intervals)) {
+      Preconditions.checkNotNull(
+          client.submit(
+              new SurrogateAction<>(supervisorTaskId, new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval))
+          ),
+          "Cannot acquire a lock for interval[%s]", interval
+      );
+    }
+  }
+
+  private SegmentAllocator createSegmentAllocator(
+      TaskToolbox toolbox,
+      ParallelIndexTaskClient taskClient,
+      ParallelIndexIngestionSpec ingestionSchema
+  )
+  {
+    final DataSchema dataSchema = ingestionSchema.getDataSchema();
+    final boolean explicitIntervals = dataSchema.getGranularitySpec().bucketIntervals().isPresent();
+    final ParallelIndexIOConfig ioConfig = ingestionSchema.getIOConfig();
+    if (ioConfig.isAppendToExisting() || !explicitIntervals) {
+      return new ActionBasedSegmentAllocator(
+          toolbox.getTaskActionClient(),
+          dataSchema,
+          (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SurrogateAction<>(
+              supervisorTaskId,
+              new SegmentAllocateAction(
+                  schema.getDataSource(),
+                  row.getTimestamp(),
+                  schema.getGranularitySpec().getQueryGranularity(),
+                  schema.getGranularitySpec().getSegmentGranularity(),
+                  sequenceName,
+                  previousSegmentId,
+                  skipSegmentLineageCheck
+              )
+          )
+      );
+    } else {
+      return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment(
+          supervisorTaskId,
+          row.getTimestamp()
+      );
+    }
+  }
+
+  /**
+   * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
+   * If there is no segment for the row, a new one is created.  Segments can be published in the middle of reading inputs
+   * if one of below conditions are satisfied.
+   *
+   * <ul>
+   * <li>
+   * If the number of rows in a segment exceeds {@link ParallelIndexTuningConfig#targetPartitionSize}
+   * </li>
+   * <li>
+   * If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link ParallelIndexTuningConfig#maxTotalRows}
+   * </li>
+   * </ul>
+   *
+   * At the end of this method, all the remaining segments are published.
+   *
+   * @return true if generated segments are successfully published, otherwise false
+   */
+  private List<DataSegment> generateAndPushSegments(
+      final TaskToolbox toolbox,
+      final ParallelIndexTaskClient taskClient,
+      final FirehoseFactory firehoseFactory,
+      final File firehoseTempDir
+  ) throws IOException, InterruptedException
+  {
+    final DataSchema dataSchema = ingestionSchema.getDataSchema();
+    final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
+    final FireDepartment fireDepartmentForMetrics = new FireDepartment(
+        dataSchema, new RealtimeIOConfig(null, null, null), null
+    );
+    final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
+
+    if (toolbox.getMonitorScheduler() != null) {
+      toolbox.getMonitorScheduler().addMonitor(
+          new RealtimeMetricsMonitor(
+              Collections.singletonList(fireDepartmentForMetrics),
+              Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()})
+          )
+      );
+    }
+
+    final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
+    final long pushTimeout = tuningConfig.getPushTimeout();
+    final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent();
+    final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient, ingestionSchema);
+
+    try (
+        final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig);
+        final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator);
+        final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir)
+    ) {
+      driver.startJob();
+
+      final List<DataSegment> pushedSegments = new ArrayList<>();
+
+      while (firehose.hasMore()) {
+        try {
+          final InputRow inputRow = firehose.nextRow();
+
+          if (inputRow == null) {
+            fireDepartmentMetrics.incrementThrownAway();
+            continue;
+          }
+
+          if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) {
+            final String errorMsg = StringUtils.format(
+                "Encountered row with timestamp that cannot be represented as a long: [%s]",
+                inputRow
+            );
+            throw new ParseException(errorMsg);
+          }
+
+          if (explicitIntervals) {
+            final Optional<Interval> optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp());
+            if (!optInterval.isPresent()) {
+              fireDepartmentMetrics.incrementThrownAway();
+              continue;
+            }
+          }
+
+          // Segments are created as needed, using a single sequence name. They may be allocated from the overlord
+          // (in append mode) or may be created on our own authority (in overwrite mode).
+          final String sequenceName = getId();
+          final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName);
+
+          if (addResult.isOk()) {
+            if (exceedMaxRowsInSegment(addResult.getNumRowsInSegment(), tuningConfig) ||
+                exceedMaxRowsInAppenderator(addResult.getTotalNumRowsInAppenderator(), tuningConfig)) {
+              // There can be some segments waiting for being published even though any rows won't be added to them.
+              // If those segments are not published here, the available space in appenderator will be kept to be small
+              // which makes the size of segments smaller.
+              final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+              pushedSegments.addAll(pushed.getSegments());
+              log.info("Pushed segments[%s]", pushed.getSegments());
+            }
+          } else {
+            throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp());
+          }
+
+          fireDepartmentMetrics.incrementProcessed();
+        }
+        catch (ParseException e) {
+          if (tuningConfig.isReportParseExceptions()) {
+            throw e;
+          } else {
+            fireDepartmentMetrics.incrementUnparseable();
+          }
+        }
+      }
+
+      final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+      pushedSegments.addAll(pushed.getSegments());
+      log.info("Pushed segments[%s]", pushed.getSegments());
+
+      return pushedSegments;
+    }
+    catch (TimeoutException | ExecutionException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static boolean exceedMaxRowsInSegment(
+      int numRowsInSegment,
+      ParallelIndexTuningConfig indexTuningConfig
+  )
+  {
+    // maxRowsInSegment should be null if numShards is set in indexTuningConfig
+    final Integer maxRowsInSegment = indexTuningConfig.getTargetPartitionSize();
+    return maxRowsInSegment != null && maxRowsInSegment <= numRowsInSegment;
+  }
+
+  private static boolean exceedMaxRowsInAppenderator(
+      long numRowsInAppenderator,
+      ParallelIndexTuningConfig indexTuningConfig
+  )
+  {
+    // maxRowsInAppenderator should be null if numShards is set in indexTuningConfig
+    final Long maxRowsInAppenderator = indexTuningConfig.getMaxTotalRows();
+    return maxRowsInAppenderator != null && maxRowsInAppenderator <= numRowsInAppenderator;
+  }
+
+  private static Appenderator newAppenderator(
+      FireDepartmentMetrics metrics,
+      TaskToolbox toolbox,
+      DataSchema dataSchema,
+      ParallelIndexTuningConfig tuningConfig
+  )
+  {
+    return Appenderators.createOffline(
+        dataSchema,
+        tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()),
+        metrics,
+        toolbox.getSegmentPusher(),
+        toolbox.getObjectMapper(),
+        toolbox.getIndexIO(),
+        toolbox.getIndexMergerV9()
+    );
+  }
+
+  private static BatchAppenderatorDriver newDriver(
+      final Appenderator appenderator,
+      final TaskToolbox toolbox,
+      final SegmentAllocator segmentAllocator
+  )
+  {
+    return new BatchAppenderatorDriver(
+        appenderator,
+        segmentAllocator,
+        new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
+        toolbox.getDataSegmentKiller()
+    );
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java
new file mode 100644
index 0000000..c1ee841
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.data.input.InputSplit;
+
+import java.util.Map;
+
+class ParallelIndexSubTaskSpec extends SubTaskSpec<ParallelIndexSubTask>
+{
+  private final ParallelIndexIngestionSpec ingestionSpec;
+
+  @JsonCreator
+  ParallelIndexSubTaskSpec(
+      String id,
+      String groupId,
+      String supervisorTaskId,
+      ParallelIndexIngestionSpec indexIngestionSpec,
+      Map<String, Object> context,
+      InputSplit inputSplit
+  )
+  {
+    super(id, groupId, supervisorTaskId, context, inputSplit);
+    this.ingestionSpec = indexIngestionSpec;
+  }
+
+  @JsonProperty
+  public ParallelIndexIngestionSpec getIngestionSpec()
+  {
+    return ingestionSpec;
+  }
+
+  @Override
+  public ParallelIndexSubTask newSubTask(int numAttempts)
+  {
+    return new ParallelIndexSubTask(
+        null,
+        getGroupId(),
+        null,
+        getSupervisorTaskId(),
+        numAttempts,
+        getIngestionSpec(),
+        getContext(),
+        null,
+        null
+    );
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
new file mode 100644
index 0000000..438e0f1
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.FirehoseFactory;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexing.common.Counters;
+import io.druid.indexing.common.TaskLock;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.LockListAction;
+import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.stats.RowIngestionMetersFactory;
+import io.druid.indexing.common.task.AbstractTask;
+import io.druid.indexing.common.task.IndexTask;
+import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
+import io.druid.indexing.common.task.IndexTask.IndexTuningConfig;
+import io.druid.indexing.common.task.IndexTaskUtils;
+import io.druid.indexing.common.task.TaskResource;
+import io.druid.indexing.common.task.Tasks;
+import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
+import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.segment.indexing.granularity.GranularitySpec;
+import io.druid.segment.realtime.appenderator.SegmentIdentifier;
+import io.druid.segment.realtime.firehose.ChatHandler;
+import io.druid.segment.realtime.firehose.ChatHandlerProvider;
+import io.druid.segment.realtime.firehose.ChatHandlers;
+import io.druid.server.security.Action;
+import io.druid.server.security.AuthorizerMapper;
+import io.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.stream.Collectors;
+
+/**
+ * ParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is
+ * applicable if the input {@link FiniteFirehoseFactory} is splittable. While this task is running, it can submit
+ * multiple child tasks to overlords. This task succeeds only when all its child tasks succeed; otherwise it fails.
+ *
+ * @see ParallelIndexTaskRunner
+ */
+public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHandler
+{
+  public static final String TYPE = "index_parallel";
+
+  private static final Logger log = new Logger(ParallelIndexSupervisorTask.class);
+
+  private final ParallelIndexIngestionSpec ingestionSchema;
+  private final FiniteFirehoseFactory<?, ?> baseFirehoseFactory;
+  private final IndexingServiceClient indexingServiceClient;
+  private final ChatHandlerProvider chatHandlerProvider;
+  private final AuthorizerMapper authorizerMapper;
+  private final RowIngestionMetersFactory rowIngestionMetersFactory;
+
+  private final Counters counters = new Counters();
+
+  private volatile ParallelIndexTaskRunner runner;
+
+  // toolbox is initlized when run() is called, and can be used for processing HTTP endpoint requests.
+  private volatile TaskToolbox toolbox;
+
+  @JsonCreator
+  public ParallelIndexSupervisorTask(
+      @JsonProperty("id") String id,
+      @JsonProperty("resource") TaskResource taskResource,
+      @JsonProperty("spec") ParallelIndexIngestionSpec ingestionSchema,
+      @JsonProperty("context") Map<String, Object> context,
+      @JacksonInject @Nullable IndexingServiceClient indexingServiceClient, // null in overlords
+      @JacksonInject @Nullable ChatHandlerProvider chatHandlerProvider,     // null in overlords
+      @JacksonInject AuthorizerMapper authorizerMapper,
+      @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory
+  )
+  {
+    super(
+        getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()),
+        null,
+        taskResource,
+        ingestionSchema.getDataSchema().getDataSource(),
+        context
+    );
+
+    this.ingestionSchema = ingestionSchema;
+
+    final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
+    if (!(firehoseFactory instanceof FiniteFirehoseFactory)) {
+      throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName());
+    }
+
+    this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory;
+    this.indexingServiceClient = indexingServiceClient;
+    this.chatHandlerProvider = chatHandlerProvider;
+    this.authorizerMapper = authorizerMapper;
+    this.rowIngestionMetersFactory = rowIngestionMetersFactory;
+
+    if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) {
+      log.warn("maxSavedParseExceptions is not supported yet");
+    }
+    if (ingestionSchema.getTuningConfig().getMaxParseExceptions() > 0) {
+      log.warn("maxParseExceptions is not supported yet");
+    }
+    if (ingestionSchema.getTuningConfig().isLogParseExceptions()) {
+      log.warn("logParseExceptions is not supported yet");
+    }
+  }
+
+  @Override
+  public int getPriority()
+  {
+    return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
+  }
+
+  @Override
+  public String getType()
+  {
+    return TYPE;
+  }
+
+  @JsonProperty("spec")
+  public ParallelIndexIngestionSpec getIngestionSchema()
+  {
+    return ingestionSchema;
+  }
+
+  @VisibleForTesting
+  @Nullable
+  ParallelIndexTaskRunner getRunner()
+  {
+    return runner;
+  }
+
+  @VisibleForTesting
+  AuthorizerMapper getAuthorizerMapper()
+  {
+    return authorizerMapper;
+  }
+
+  @VisibleForTesting
+  ParallelIndexTaskRunner createRunner(TaskToolbox toolbox)
+  {
+    if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) {
+      throw new UnsupportedOperationException("Perfect roll-up is not supported yet");
+    } else {
+      runner = new SinglePhaseParallelIndexTaskRunner(
+          toolbox,
+          getId(),
+          getGroupId(),
+          ingestionSchema,
+          getContext(),
+          indexingServiceClient
+      );
+    }
+    return runner;
+  }
+
+  @VisibleForTesting
+  void setRunner(ParallelIndexTaskRunner runner)
+  {
+    this.runner = runner;
+  }
+
+  @Override
+  public boolean isReady(TaskActionClient taskActionClient) throws Exception
+  {
+    final Optional<SortedSet<Interval>> intervals = ingestionSchema.getDataSchema()
+                                                                   .getGranularitySpec()
+                                                                   .bucketIntervals();
+
+    return !intervals.isPresent() || isReady(taskActionClient, intervals.get());
+  }
+
+  static boolean isReady(TaskActionClient actionClient, SortedSet<Interval> intervals) throws IOException
+  {
+    final List<TaskLock> locks = getTaskLocks(actionClient);
+    if (locks.isEmpty()) {
+      try {
+        Tasks.tryAcquireExclusiveLocks(actionClient, intervals);
+      }
+      catch (Exception e) {
+        log.error(e, "Failed to acquire locks for intervals[%s]", intervals);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public TaskStatus run(TaskToolbox toolbox) throws Exception
+  {
+    setToolbox(toolbox);
+
+    log.info(
+        "Found chat handler of class[%s]",
+        Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName()
+    );
+    chatHandlerProvider.register(getId(), this, false);
+
+    try {
+      if (baseFirehoseFactory.isSplittable()) {
+        return runParallel(toolbox);
+      } else {
+        log.warn(
+            "firehoseFactory[%s] is not splittable. Running sequentially",
+            baseFirehoseFactory.getClass().getSimpleName()
+        );
+        return runSequential(toolbox);
+      }
+    }
+    finally {
+      chatHandlerProvider.unregister(getId());
+    }
+  }
+
+  @VisibleForTesting
+  void setToolbox(TaskToolbox toolbox)
+  {
+    this.toolbox = toolbox;
+  }
+
+  private TaskStatus runParallel(TaskToolbox toolbox) throws Exception
+  {
+    createRunner(toolbox);
+    return TaskStatus.fromCode(getId(), runner.run());
+  }
+
+  private TaskStatus runSequential(TaskToolbox toolbox) throws Exception
+  {
+    return new IndexTask(
+        getId(),
+        getGroupId(),
+        getTaskResource(),
+        getDataSource(),
+        new IndexIngestionSpec(
+            getIngestionSchema().getDataSchema(),
+            getIngestionSchema().getIOConfig(),
+            convertToIndexTuningConfig(getIngestionSchema().getTuningConfig())
+        ),
+        getContext(),
+        authorizerMapper,
+        chatHandlerProvider,
+        rowIngestionMetersFactory
+    ).run(toolbox);
+  }
+
+  private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningConfig tuningConfig)
+  {
+    return new IndexTuningConfig(
+        tuningConfig.getTargetPartitionSize(),
+        tuningConfig.getMaxRowsInMemory(),
+        tuningConfig.getMaxBytesInMemory(),
+        tuningConfig.getMaxTotalRows(),
+        null,
+        tuningConfig.getNumShards(),
+        tuningConfig.getIndexSpec(),
+        tuningConfig.getMaxPendingPersists(),
+        true,
+        tuningConfig.isForceExtendableShardSpecs(),
+        false,
+        tuningConfig.isReportParseExceptions(),
+        null,
+        tuningConfig.getPushTimeout(),
+        tuningConfig.getSegmentWriteOutMediumFactory(),
+        tuningConfig.isLogParseExceptions(),
+        tuningConfig.getMaxParseExceptions(),
+        tuningConfig.getMaxSavedParseExceptions()
+    );
+  }
+
+  // Internal APIs
+
+  /**
+   * Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}.
+   * The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec})
+   * per bucket interval.
+   */
+  @POST
+  @Path("/segment/allocate")
+  @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE)
+  public Response allocateSegment(
+      DateTime timestamp,
+      @Context final HttpServletRequest req
+  )
+  {
+    ChatHandlers.authorizationCheck(
+        req,
+        Action.READ,
+        getDataSource(),
+        authorizerMapper
+    );
+
+    if (toolbox == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    }
+
+    try {
+      final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp);
+      return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build();
+    }
+    catch (IOException | IllegalStateException e) {
+      return Response.serverError().entity(Throwables.getStackTraceAsString(e)).build();
+    }
+    catch (IllegalArgumentException e) {
+      return Response.status(Response.Status.BAD_REQUEST).entity(Throwables.getStackTraceAsString(e)).build();
+    }
+  }
+
+  @VisibleForTesting
+  SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException
+  {
+    final String dataSource = getDataSource();
+    final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec();
+    final SortedSet<Interval> bucketIntervals = Preconditions.checkNotNull(
+        granularitySpec.bucketIntervals().orNull(),
+        "bucketIntervals"
+    );
+    // List locks whenever allocating a new segment because locks might be revoked and no longer valid.
+    final Map<Interval, String> versions = toolbox
+        .getTaskActionClient()
+        .submit(new LockListAction())
+        .stream()
+        .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
+
+    final Optional<Interval> maybeInterval = granularitySpec.bucketInterval(timestamp);
+    if (!maybeInterval.isPresent()) {
+      throw new IAE("Could not find interval for timestamp [%s]", timestamp);
+    }
+
+    final Interval interval = maybeInterval.get();
+    if (!bucketIntervals.contains(interval)) {
+      throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec);
+    }
+
+    final int partitionNum = counters.increment(interval.toString(), 1);
+    return new SegmentIdentifier(
+        dataSource,
+        interval,
+        findVersion(versions, interval),
+        new NumberedShardSpec(partitionNum, 0)
+    );
+  }
+
+  private static String findVersion(Map<Interval, String> versions, Interval interval)
+  {
+    return versions.entrySet().stream()
+                   .filter(entry -> entry.getKey().contains(interval))
+                   .map(Entry::getValue)
+                   .findFirst()
+                   .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
+  }
+
+  /**
+   * {@link ParallelIndexSubTask}s call this API to report the segments they've generated and pushed.
+   */
+  @POST
+  @Path("/report")
+  @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE)
+  public Response report(
+      PushedSegmentsReport report,
+      @Context final HttpServletRequest req
+  )
+  {
+    ChatHandlers.authorizationCheck(
+        req,
+        Action.WRITE,
+        getDataSource(),
+        authorizerMapper
+    );
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      runner.collectReport(report);
+      return Response.ok().build();
+    }
+  }
+
+  // External APIs to get running status
+
+  @GET
+  @Path("/mode")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getMode(@Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build();
+    }
+  }
+
+  @GET
+  @Path("/progress")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getProgress(@Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      return Response.ok(runner.getProgress()).build();
+    }
+  }
+
+  @GET
+  @Path("/subtasks/running")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getRunningTasks(@Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      return Response.ok(runner.getRunningTaskIds()).build();
+    }
+  }
+
+  @GET
+  @Path("/subtaskspecs")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getSubTaskSpecs(@Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      return Response.ok(runner.getSubTaskSpecs()).build();
+    }
+  }
+
+  @GET
+  @Path("/subtaskspecs/running")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      return Response.ok(runner.getRunningSubTaskSpecs()).build();
+    }
+  }
+
+  @GET
+  @Path("/subtaskspecs/complete")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      return Response.ok(runner.getCompleteSubTaskSpecs()).build();
+    }
+  }
+
+  @GET
+  @Path("/subtaskspec/{id}")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      final SubTaskSpec subTaskSpec = runner.getSubTaskSpec(id);
+      if (subTaskSpec == null) {
+        return Response.status(Response.Status.NOT_FOUND).build();
+      } else {
+        return Response.ok(subTaskSpec).build();
+      }
+    }
+  }
+
+  @GET
+  @Path("/subtaskspec/{id}/state")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getSubTaskState(@PathParam("id") String id, @Context final HttpServletRequest req)
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      final SubTaskSpecStatus subTaskSpecStatus = runner.getSubTaskState(id);
+      if (subTaskSpecStatus == null) {
+        return Response.status(Response.Status.NOT_FOUND).build();
+      } else {
+        return Response.ok(subTaskSpecStatus).build();
+      }
+    }
+  }
+
+  @GET
+  @Path("/subtaskspec/{id}/history")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getCompleteSubTaskSpecAttemptHistory(
+      @PathParam("id") String id,
+      @Context final HttpServletRequest req
+  )
+  {
+    IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
+    if (runner == null) {
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
+    } else {
+      final TaskHistory taskHistory = runner.getCompleteSubTaskSpecAttemptHistory(id);
+      if (taskHistory == null) {
+        return Response.status(Status.NOT_FOUND).build();
+      } else {
+        return Response.ok(taskHistory.getAttemptHistory()).build();
+      }
+    }
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java
new file mode 100644
index 0000000..6e11160
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.indexing.common.IndexTaskClient;
+import io.druid.indexing.common.TaskInfoProvider;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.response.FullResponseHolder;
+import io.druid.segment.realtime.appenderator.SegmentIdentifier;
+import io.druid.timeline.DataSegment;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import java.io.IOException;
+import java.util.List;
+
+public class ParallelIndexTaskClient extends IndexTaskClient
+{
+  private final String subtaskId;
+
+  ParallelIndexTaskClient(
+      HttpClient httpClient,
+      ObjectMapper objectMapper,
+      TaskInfoProvider taskInfoProvider,
+      Duration httpTimeout,
+      String callerId,
+      long numRetries
+  )
+  {
+    super(httpClient, objectMapper, taskInfoProvider, httpTimeout, callerId, 1, numRetries);
+    this.subtaskId = callerId;
+  }
+
+  String getSubtaskId()
+  {
+    return subtaskId;
+  }
+
+  public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
+  {
+    final FullResponseHolder response = submitSmileRequest(
+        supervisorTaskId,
+        HttpMethod.POST,
+        "segment/allocate",
+        null,
+        serialize(timestamp),
+        true
+    );
+    if (!isSuccess(response)) {
+      throw new ISE(
+          "task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]",
+          supervisorTaskId,
+          response.getStatus().getCode(),
+          response.getContent()
+      );
+    } else {
+      return deserialize(
+          response.getContent(),
+          new TypeReference<SegmentIdentifier>()
+          {
+          }
+      );
+    }
+  }
+
+  public void report(String supervisorTaskId, List<DataSegment> pushedSegments)
+  {
+    try {
+      final FullResponseHolder response = submitSmileRequest(
+          supervisorTaskId,
+          HttpMethod.POST,
+          "report",
+          null,
+          serialize(new PushedSegmentsReport(subtaskId, pushedSegments)),
+          true
+      );
+      if (!isSuccess(response)) {
+        throw new ISE(
+            "Failed to send taskReports to task[%s] with the HTTP code [%d]",
+            supervisorTaskId,
+            response.getStatus().getCode()
+        );
+      }
+    }
+    catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java
similarity index 68%
copy from extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
copy to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java
index 0d88a15..645011f 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java
@@ -17,46 +17,49 @@
  * under the License.
  */
 
-package io.druid.indexing.kafka;
+package io.druid.indexing.common.task.batch.parallel;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
-import io.druid.java.util.http.client.HttpClient;
 import io.druid.guice.annotations.EscalatedGlobal;
-import io.druid.guice.annotations.Json;
+import io.druid.guice.annotations.Smile;
 import io.druid.indexing.common.TaskInfoProvider;
+import io.druid.indexing.common.task.IndexTaskClientFactory;
+import io.druid.java.util.http.client.HttpClient;
 import org.joda.time.Duration;
 
-public class KafkaIndexTaskClientFactory
+public class ParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexTaskClient>
 {
-  private HttpClient httpClient;
-  private ObjectMapper mapper;
+  private final HttpClient httpClient;
+  private final ObjectMapper mapper;
 
   @Inject
-  public KafkaIndexTaskClientFactory(
+  public ParallelIndexTaskClientFactory(
       @EscalatedGlobal HttpClient httpClient,
-      @Json ObjectMapper mapper
+      @Smile ObjectMapper mapper
   )
   {
     this.httpClient = httpClient;
     this.mapper = mapper;
   }
 
-  public KafkaIndexTaskClient build(
+  @Override
+  public ParallelIndexTaskClient build(
       TaskInfoProvider taskInfoProvider,
-      String dataSource,
+      String callerId,
       int numThreads,
       Duration httpTimeout,
       long numRetries
   )
   {
-    return new KafkaIndexTaskClient(
+    Preconditions.checkState(numThreads == 1, "expect numThreads to be 1");
+    return new ParallelIndexTaskClient(
         httpClient,
         mapper,
         taskInfoProvider,
-        dataSource,
-        numThreads,
         httpTimeout,
+        callerId,
         numRetries
     );
   }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java
new file mode 100644
index 0000000..a807741
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.task.Task;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * ParallelIndexTaskRunner is the actual task runner of {@link ParallelIndexSupervisorTask}. There is currently a single
+ * implementation, i.e. {@link SinglePhaseParallelIndexTaskRunner} which supports only best-effort roll-up. We can add
+ * more implementations for different distributed indexing algorithms in the future.
+ */
+public interface ParallelIndexTaskRunner<T extends Task>
+{
+  /**
+   * Runs the task.
+   */
+  TaskState run() throws Exception;
+
+  /**
+   * {@link PushedSegmentsReport} is the report sent by {@link ParallelIndexSubTask}s. The subTasks call this method to
+   * send their reports after pushing generated segments to deep storage.
+   */
+  void collectReport(PushedSegmentsReport report);
+
+  /**
+   * Returns the current {@link ParallelIndexingProgress}.
+   */
+  ParallelIndexingProgress getProgress();
+
+  /**
+   * Returns the IDs of current running tasks.
+   */
+  Set<String> getRunningTaskIds();
+
+  /**
+   * Returns all {@link SubTaskSpec}s.
+   */
+  List<SubTaskSpec<T>> getSubTaskSpecs();
+
+  /**
+   * Returns running {@link SubTaskSpec}s. A {@link SubTaskSpec} is running if there is a running {@link Task} created
+   * using that subTaskSpec.
+   *
+   * @see SubTaskSpec#newSubTask
+   */
+  List<SubTaskSpec<T>> getRunningSubTaskSpecs();
+
+  /**
+   * Returns complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if there is a succeeded or failed
+   * {@link Task} created using that subTaskSpec.
+   *
+   * @see SubTaskSpec#newSubTask
+   */
+  List<SubTaskSpec<T>> getCompleteSubTaskSpecs();
+
+  /**
+   * Returns the {@link SubTaskSpec} of the given ID or null if it's not found.
+   */
+  @Nullable
+  SubTaskSpec<T> getSubTaskSpec(String subTaskSpecId);
+
+  /**
+   * Returns {@link SubTaskSpecStatus} of the given ID or null if it's not found.
+   */
+  @Nullable
+  SubTaskSpecStatus getSubTaskState(String subTaskSpecId);
+
+  /**
+   * Returns {@link TaskHistory} of the given ID or null if it's not found.
+   */
+  @Nullable
+  TaskHistory<T> getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId);
+
+  class SubTaskSpecStatus
+  {
+    private final ParallelIndexSubTaskSpec spec;
+    @Nullable
+    private final TaskStatusPlus currentStatus; // null if there is no running task for the spec
+    private final List<TaskStatusPlus> taskHistory; // can be empty if there is no history
+
+    @JsonCreator
+    public SubTaskSpecStatus(
+        @JsonProperty("spec") ParallelIndexSubTaskSpec spec,
+        @JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus,
+        @JsonProperty("taskHistory") List<TaskStatusPlus> taskHistory
+    )
+    {
+      this.spec = spec;
+      this.currentStatus = currentStatus;
+      this.taskHistory = taskHistory;
+    }
+
+    @JsonProperty
+    public ParallelIndexSubTaskSpec getSpec()
+    {
+      return spec;
+    }
+
+    @JsonProperty
+    @Nullable
+    public TaskStatusPlus getCurrentStatus()
+    {
+      return currentStatus;
+    }
+
+    @JsonProperty
+    public List<TaskStatusPlus> getTaskHistory()
+    {
+      return taskHistory;
+    }
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java
new file mode 100644
index 0000000..b9a57e2
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import io.druid.indexing.common.task.IndexTask.IndexTuningConfig;
+import io.druid.segment.IndexSpec;
+import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
+import org.joda.time.Duration;
+import org.joda.time.Period;
+
+import javax.annotation.Nullable;
+import java.util.Objects;
+
+@JsonTypeName("index_parallel")
+public class ParallelIndexTuningConfig extends IndexTuningConfig
+{
+  private static final int DEFAULT_MAX_NUM_BATCH_TASKS = Integer.MAX_VALUE; // unlimited
+  private static final int DEFAULT_MAX_RETRY = 3;
+  private static final long DEFAULT_TASK_STATUS_CHECK_PERIOD_MS = 1000;
+
+  private static final Duration DEFAULT_CHAT_HANDLER_TIMEOUT = new Period("PT10S").toStandardDuration();
+  private static final int DEFAULT_CHAT_HANDLER_NUM_RETRIES = 5;
+
+  private final int maxNumSubTasks;
+  private final int maxRetry;
+  private final long taskStatusCheckPeriodMs;
+
+  private final Duration chatHandlerTimeout;
+  private final int chatHandlerNumRetries;
+
+  public static ParallelIndexTuningConfig defaultConfig()
+  {
+    return new ParallelIndexTuningConfig(
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null
+    );
+  }
+
+  @JsonCreator
+  public ParallelIndexTuningConfig(
+      @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize,
+      @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
+      @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory,
+      @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows,
+      @JsonProperty("numShards") @Nullable Integer numShards,
+      @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
+      @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists,
+      @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs,
+      @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup,
+      @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions,
+      @JsonProperty("pushTimeout") @Nullable Long pushTimeout,
+      @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
+      @JsonProperty("maxNumSubTasks") @Nullable Integer maxNumSubTasks,
+      @JsonProperty("maxRetry") @Nullable Integer maxRetry,
+      @JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs,
+      @JsonProperty("chatHandlerTimeout") @Nullable Duration chatHandlerTimeout,
+      @JsonProperty("chatHandlerNumRetries") @Nullable Integer chatHandlerNumRetries,
+      @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
+      @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
+      @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions
+  )
+  {
+    super(
+        targetPartitionSize,
+        maxRowsInMemory,
+        maxBytesInMemory,
+        maxTotalRows,
+        null,
+        numShards,
+        indexSpec,
+        maxPendingPersists,
+        null,
+        forceExtendableShardSpecs,
+        forceGuaranteedRollup,
+        reportParseExceptions,
+        null,
+        pushTimeout,
+        segmentWriteOutMediumFactory,
+        logParseExceptions,
+        maxParseExceptions,
+        maxSavedParseExceptions
+    );
+
+    this.maxNumSubTasks = maxNumSubTasks == null ? DEFAULT_MAX_NUM_BATCH_TASKS : maxNumSubTasks;
+    this.maxRetry = maxRetry == null ? DEFAULT_MAX_RETRY : maxRetry;
+    this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs == null ?
+                                   DEFAULT_TASK_STATUS_CHECK_PERIOD_MS :
+                                   taskStatusCheckPeriodMs;
+
+    this.chatHandlerTimeout = DEFAULT_CHAT_HANDLER_TIMEOUT;
+    this.chatHandlerNumRetries = DEFAULT_CHAT_HANDLER_NUM_RETRIES;
+  }
+
+  @JsonProperty
+  public int getMaxNumSubTasks()
+  {
+    return maxNumSubTasks;
+  }
+
+  @JsonProperty
+  public int getMaxRetry()
+  {
+    return maxRetry;
+  }
+
+  @JsonProperty
+  public long getTaskStatusCheckPeriodMs()
+  {
+    return taskStatusCheckPeriodMs;
+  }
+
+  @JsonProperty
+  public Duration getChatHandlerTimeout()
+  {
+    return chatHandlerTimeout;
+  }
+
+  @JsonProperty
+  public int getChatHandlerNumRetries()
+  {
+    return chatHandlerNumRetries;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    ParallelIndexTuningConfig that = (ParallelIndexTuningConfig) o;
+    return maxNumSubTasks == that.maxNumSubTasks &&
+           maxRetry == that.maxRetry &&
+           taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs &&
+           chatHandlerNumRetries == that.chatHandlerNumRetries &&
+           Objects.equals(chatHandlerTimeout, that.chatHandlerTimeout);
+  }
+
+  @Override
+  public int hashCode()
+  {
+
+    return Objects.hash(
+        super.hashCode(),
+        maxNumSubTasks,
+        maxRetry,
+        taskStatusCheckPeriodMs,
+        chatHandlerTimeout,
+        chatHandlerNumRetries
+    );
+  }
+}
diff --git a/api/src/main/java/io/druid/indexer/RunnerTaskState.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java
similarity index 76%
copy from api/src/main/java/io/druid/indexer/RunnerTaskState.java
copy to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java
index a834491..bb813c7 100644
--- a/api/src/main/java/io/druid/indexer/RunnerTaskState.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package io.druid.indexer;
+package io.druid.indexing.common.task.batch.parallel;
+
 /**
- * This includes the state of a task in the task runner not covered by {@link TaskState}, this state is not stored in database
+ * Represents the current progress of {@link ParallelIndexSupervisorTask}. Implementations can be different depending on
+ * the distributed indexing algorithm.
  */
-public enum RunnerTaskState
+interface ParallelIndexingProgress
 {
-  WAITING,
-  PENDING,
-  RUNNING,
-  NONE; // is used for a completed task
 }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java
new file mode 100644
index 0000000..548d187
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import io.druid.timeline.DataSegment;
+
+import java.util.List;
+
+/**
+ * This class is used in native parallel batch indexing, currently only in {@link SinglePhaseParallelIndexTaskRunner}.
+ * In native parallel batch indexing, each subTask generates and pushes segments and sends a report to the
+ * supervisorTask. Once the supervisorTask collects all reports, it publishes all the pushed segments at once.
+ */
+public class PushedSegmentsReport
+{
+  private final String taskId;
+  private final List<DataSegment> segments;
+
+  @JsonCreator
+  public PushedSegmentsReport(
+      @JsonProperty("taskId") String taskId,
+      @JsonProperty("segments") List<DataSegment> segments
+  )
+  {
+    this.taskId = Preconditions.checkNotNull(taskId, "taskId");
+    this.segments = Preconditions.checkNotNull(segments, "segments");
+  }
+
+  @JsonProperty
+  public String getTaskId()
+  {
+    return taskId;
+  }
+
+  @JsonProperty
+  public List<DataSegment> getSegments()
+  {
+    return segments;
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java
new file mode 100644
index 0000000..ae1addf
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java
@@ -0,0 +1,484 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.FirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
+import io.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry;
+import io.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.segment.realtime.appenderator.SegmentIdentifier;
+import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
+import io.druid.segment.realtime.appenderator.UsedSegmentChecker;
+import io.druid.timeline.DataSegment;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * An implementation of {@link ParallelIndexTaskRunner} to support best-effort roll-up. This runner can submit and
+ * monitor multiple {@link ParallelIndexSubTask}s.
+ *
+ * As its name indicates, distributed indexing is done in a single phase, i.e., without shuffling intermediate data. As
+ * a result, this task can't be used for perfect rollup.
+ */
+public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunner<ParallelIndexSubTask>
+{
+  private static final Logger log = new Logger(SinglePhaseParallelIndexTaskRunner.class);
+
+  private final TaskToolbox toolbox;
+  private final String taskId;
+  private final String groupId;
+  private final ParallelIndexIngestionSpec ingestionSchema;
+  private final Map<String, Object> context;
+  private final FiniteFirehoseFactory<?, ?> baseFirehoseFactory;
+  private final int maxNumTasks;
+  private final IndexingServiceClient indexingServiceClient;
+
+  private final BlockingQueue<SubTaskCompleteEvent<ParallelIndexSubTask>> taskCompleteEvents =
+      new LinkedBlockingDeque<>();
+
+  // subTaskId -> report
+  private final ConcurrentMap<String, PushedSegmentsReport> segmentsMap = new ConcurrentHashMap<>();
+
+  private volatile boolean stopped;
+  private volatile TaskMonitor<ParallelIndexSubTask> taskMonitor;
+
+  private int nextSpecId = 0;
+
+  SinglePhaseParallelIndexTaskRunner(
+      TaskToolbox toolbox,
+      String taskId,
+      String groupId,
+      ParallelIndexIngestionSpec ingestionSchema,
+      Map<String, Object> context,
+      IndexingServiceClient indexingServiceClient
+  )
+  {
+    this.toolbox = toolbox;
+    this.taskId = taskId;
+    this.groupId = groupId;
+    this.ingestionSchema = ingestionSchema;
+    this.context = context;
+    this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory();
+    this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumSubTasks();
+    this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient");
+  }
+
+  @Override
+  public TaskState run() throws Exception
+  {
+    final Iterator<ParallelIndexSubTaskSpec> subTaskSpecIterator = subTaskSpecIterator().iterator();
+    final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs();
+
+    taskMonitor = new TaskMonitor<>(
+        Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"),
+        ingestionSchema.getTuningConfig().getMaxRetry(),
+        baseFirehoseFactory.getNumSplits()
+    );
+    TaskState state = TaskState.RUNNING;
+
+    taskMonitor.start(taskStatusCheckingPeriod);
+
+    try {
+      log.info("Submitting initial tasks");
+      // Submit initial tasks
+      while (isRunning() && subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) {
+        submitNewTask(taskMonitor, subTaskSpecIterator.next());
+      }
+
+      log.info("Waiting for subTasks to be completed");
+      while (isRunning()) {
+        final SubTaskCompleteEvent<ParallelIndexSubTask> taskCompleteEvent = taskCompleteEvents.poll(
+            taskStatusCheckingPeriod,
+            TimeUnit.MILLISECONDS
+        );
+
+        if (taskCompleteEvent != null) {
+          final TaskState completeState = taskCompleteEvent.getLastState();
+          switch (completeState) {
+            case SUCCESS:
+              final TaskStatusPlus completeStatus = taskCompleteEvent.getLastStatus();
+              if (completeStatus == null) {
+                throw new ISE("Last status of complete task is missing!");
+              }
+              // Pushed segments of complete tasks are supposed to be already reported.
+              if (!segmentsMap.containsKey(completeStatus.getId())) {
+                throw new ISE("Missing reports from task[%s]!", completeStatus.getId());
+              }
+
+              if (!subTaskSpecIterator.hasNext()) {
+                // We have no more subTasks to run
+                if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) {
+                  stopped = true;
+                  if (taskMonitor.isSucceeded()) {
+                    // Publishing all segments reported so far
+                    publish(toolbox);
+
+                    // Succeeded
+                    state = TaskState.SUCCESS;
+                  } else {
+                    // Failed
+                    final SinglePhaseParallelIndexingProgress monitorStatus = taskMonitor.getProgress();
+                    throw new ISE(
+                        "Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks",
+                        monitorStatus.getExpectedSucceeded(),
+                        monitorStatus.getSucceeded(),
+                        monitorStatus.getFailed()
+                    );
+                  }
+                }
+              } else if (taskMonitor.getNumRunningTasks() < maxNumTasks) {
+                // We have more subTasks to run
+                submitNewTask(taskMonitor, subTaskSpecIterator.next());
+              } else {
+                // We have more subTasks to run, but don't have enough available task slots
+                // do nothing
+              }
+              break;
+            case FAILED:
+              // TaskMonitor already tried everything it can do for failed tasks. We failed.
+              state = TaskState.FAILED;
+              stopped = true;
+              final TaskStatusPlus lastStatus = taskCompleteEvent.getLastStatus();
+              if (lastStatus != null) {
+                log.error("Failed because of the failed sub task[%s]", lastStatus.getId());
+              } else {
+                final ParallelIndexSubTaskSpec spec =
+                    (ParallelIndexSubTaskSpec) taskCompleteEvent.getSpec();
+                log.error(
+                    "Failed to run sub tasks for inputSplits[%s]",
+                    getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory())
+                );
+              }
+              break;
+            default:
+              throw new ISE("spec[%s] is in an invalid state[%s]", taskCompleteEvent.getSpec().getId(), completeState);
+          }
+        }
+      }
+    }
+    finally {
+      log.info("Cleaning up resources");
+      // Cleanup resources
+      taskCompleteEvents.clear();
+      taskMonitor.stop();
+
+      if (state != TaskState.SUCCESS) {
+        log.info(
+            "This task is finished with [%s] state. Killing [%d] remaining subtasks.",
+            state,
+            taskMonitor.getNumRunningTasks()
+        );
+        // if this fails, kill all sub tasks
+        // Note: this doesn't work when this task is killed by users. We need a way for gracefully shutting down tasks
+        // for resource cleanup.
+        taskMonitor.killAll();
+      }
+    }
+
+    return state;
+  }
+
+  private boolean isRunning()
+  {
+    return !stopped && !Thread.currentThread().isInterrupted();
+  }
+
+  @VisibleForTesting
+  TaskToolbox getToolbox()
+  {
+    return toolbox;
+  }
+
+  @VisibleForTesting
+  ParallelIndexIngestionSpec getIngestionSchema()
+  {
+    return ingestionSchema;
+  }
+
+  @Override
+  public void collectReport(PushedSegmentsReport report)
+  {
+    // subTasks might send their reports multiple times because of the HTTP retry.
+    // Here, we simply make sure the current report is exactly same with the previous one.
+    segmentsMap.compute(report.getTaskId(), (taskId, prevReport) -> {
+      if (prevReport != null) {
+        Preconditions.checkState(
+            prevReport.getSegments().equals(report.getSegments()),
+            "task[%s] sent two or more reports and previous report[%s] is different from the current one[%s]",
+            taskId,
+            prevReport,
+            report
+        );
+      }
+      return report;
+    });
+  }
+
+  @Override
+  public SinglePhaseParallelIndexingProgress getProgress()
+  {
+    return taskMonitor == null ? SinglePhaseParallelIndexingProgress.notRunning() : taskMonitor.getProgress();
+  }
+
+  @Override
+  public Set<String> getRunningTaskIds()
+  {
+    return taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds();
+  }
+
+  @Override
+  public List<SubTaskSpec<ParallelIndexSubTask>> getSubTaskSpecs()
+  {
+    if (taskMonitor != null) {
+      final List<SubTaskSpec<ParallelIndexSubTask>> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs();
+      final List<SubTaskSpec<ParallelIndexSubTask>> completeSubTaskSpecs = taskMonitor
+          .getCompleteSubTaskSpecs();
+      // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and
+      // completeSubTaskSpecs.
+      final Map<String, SubTaskSpec<ParallelIndexSubTask>> subTaskSpecMap = new HashMap<>(
+          runningSubTaskSpecs.size() + completeSubTaskSpecs.size()
+      );
+      runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec));
+      completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec));
+      return new ArrayList<>(subTaskSpecMap.values());
+    } else {
+      return Collections.emptyList();
+    }
+  }
+
+  @Override
+  public List<SubTaskSpec<ParallelIndexSubTask>> getRunningSubTaskSpecs()
+  {
+    return taskMonitor == null ? Collections.emptyList() : taskMonitor.getRunningSubTaskSpecs();
+  }
+
+  @Override
+  public List<SubTaskSpec<ParallelIndexSubTask>> getCompleteSubTaskSpecs()
+  {
+    return taskMonitor == null ? Collections.emptyList() : taskMonitor.getCompleteSubTaskSpecs();
+  }
+
+  @Nullable
+  @Override
+  public SubTaskSpec<ParallelIndexSubTask> getSubTaskSpec(String subTaskSpecId)
+  {
+    if (taskMonitor != null) {
+      // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after
+      // adding them to taskHistory.
+      final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntry(subTaskSpecId);
+      final TaskHistory<ParallelIndexSubTask> taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId);
+      final SubTaskSpec<ParallelIndexSubTask> subTaskSpec;
+
+      if (monitorEntry != null) {
+        subTaskSpec = monitorEntry.getSpec();
+      } else {
+        if (taskHistory != null) {
+          subTaskSpec = taskHistory.getSpec();
+        } else {
+          subTaskSpec = null;
+        }
+      }
+
+      return subTaskSpec;
+    } else {
+      return null;
+    }
+  }
+
+  @Nullable
+  @Override
+  public SubTaskSpecStatus getSubTaskState(String subTaskSpecId)
+  {
+    if (taskMonitor == null) {
+      return null;
+    } else {
+      // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after
+      // adding them to taskHistory.
+      final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntry(subTaskSpecId);
+      final TaskHistory<ParallelIndexSubTask> taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId);
+
+      final SubTaskSpecStatus subTaskSpecStatus;
+
+      if (monitorEntry != null) {
+        subTaskSpecStatus = new SubTaskSpecStatus(
+            (ParallelIndexSubTaskSpec) monitorEntry.getSpec(),
+            monitorEntry.getRunningStatus(),
+            monitorEntry.getTaskHistory()
+        );
+      } else {
+        if (taskHistory != null && !taskHistory.isEmpty()) {
+          subTaskSpecStatus = new SubTaskSpecStatus(
+              (ParallelIndexSubTaskSpec) taskHistory.getSpec(),
+              null,
+              taskHistory.getAttemptHistory()
+          );
+        } else {
+          subTaskSpecStatus = null;
+        }
+      }
+
+      return subTaskSpecStatus;
+    }
+  }
+
+  @Nullable
+  @Override
+  public TaskHistory<ParallelIndexSubTask> getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId)
+  {
+    if (taskMonitor == null) {
+      return null;
+    } else {
+      return taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId);
+    }
+  }
+
+  private void publish(TaskToolbox toolbox) throws IOException
+  {
+    final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
+      final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments);
+      return toolbox.getTaskActionClient().submit(action).isSuccess();
+    };
+    final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient());
+    final Set<DataSegment> segmentsToPublish = segmentsMap
+        .values()
+        .stream()
+        .flatMap(report -> report.getSegments().stream())
+        .collect(Collectors.toSet());
+    final boolean published = publisher.publishSegments(segmentsToPublish, null);
+
+    if (published) {
+      log.info("Published segments");
+    } else {
+      log.info("Transaction failure while publishing segments, checking if someone else beat us to it.");
+      final Set<SegmentIdentifier> segmentsIdentifiers = segmentsMap
+          .values()
+          .stream()
+          .flatMap(report -> report.getSegments().stream())
+          .map(SegmentIdentifier::fromDataSegment)
+          .collect(Collectors.toSet());
+      if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers)
+                            .equals(segmentsToPublish)) {
+        log.info("Our segments really do exist, awaiting handoff.");
+      } else {
+        throw new ISE("Failed to publish segments[%s]", segmentsToPublish);
+      }
+    }
+  }
+
+  private void submitNewTask(
+      TaskMonitor<ParallelIndexSubTask> taskMonitor,
+      ParallelIndexSubTaskSpec spec
+  )
+  {
+    log.info("Submit a new task for spec[%s] and inputSplit[%s]", spec.getId(), spec.getInputSplit());
+    final ListenableFuture<SubTaskCompleteEvent<ParallelIndexSubTask>> future = taskMonitor.submit(spec);
+    Futures.addCallback(
+        future,
+        new FutureCallback<SubTaskCompleteEvent<ParallelIndexSubTask>>()
+        {
+          @Override
+          public void onSuccess(SubTaskCompleteEvent<ParallelIndexSubTask> completeEvent)
+          {
+            // this callback is called if a task completed wheter it succeeded or not.
+            taskCompleteEvents.offer(completeEvent);
+          }
+
+          @Override
+          public void onFailure(Throwable t)
+          {
+            // this callback is called only when there were some problems in TaskMonitor.
+            log.error(t, "Error while running a task for subTaskSpec[%s]", spec);
+            taskCompleteEvents.offer(SubTaskCompleteEvent.fail(spec, t));
+          }
+        }
+    );
+  }
+
+  @VisibleForTesting
+  int getAndIncrementNextSpecId()
+  {
+    return nextSpecId++;
+  }
+
+  @VisibleForTesting
+  Stream<ParallelIndexSubTaskSpec> subTaskSpecIterator() throws IOException
+  {
+    return baseFirehoseFactory.getSplits().map(this::newTaskSpec);
+  }
+
+  @VisibleForTesting
+  ParallelIndexSubTaskSpec newTaskSpec(InputSplit split)
+  {
+    return new ParallelIndexSubTaskSpec(
+        taskId + "_" + getAndIncrementNextSpecId(),
+        groupId,
+        taskId,
+        new ParallelIndexIngestionSpec(
+            ingestionSchema.getDataSchema(),
+            new ParallelIndexIOConfig(
+                baseFirehoseFactory.withSplit(split),
+                ingestionSchema.getIOConfig().isAppendToExisting()
+            ),
+            ingestionSchema.getTuningConfig()
+        ),
+        context,
+        split
+    );
+  }
+
+  private static List<InputSplit> getSplitsIfSplittable(FirehoseFactory firehoseFactory) throws IOException
+  {
+    if (firehoseFactory instanceof FiniteFirehoseFactory) {
+      final FiniteFirehoseFactory<?, ?> finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory;
+      return finiteFirehoseFactory.getSplits().collect(Collectors.toList());
+    } else {
+      throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName());
+    }
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java
new file mode 100644
index 0000000..aa21735
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Represents the current progress of {@link SinglePhaseParallelIndexTaskRunner}.
+ */
+class SinglePhaseParallelIndexingProgress implements ParallelIndexingProgress
+{
+  /**
+   * Number of running tasks.
+   */
+  private final int running;
+
+  /**
+   * Number of succeeded tasks.
+   */
+  private final int succeeded;
+
+  /**
+   * Number of failed tasks.
+   */
+  private final int failed;
+
+  /**
+   * Number of complete (succeeded + failed) tasks.
+   */
+  private final int complete;
+
+  /**
+   * Number of total (running + complete) tasks.
+   */
+  private final int total;
+
+  /**
+   * Number of succeeded tasks for {@link SinglePhaseParallelIndexTaskRunner} to succeed.
+   */
+  private final int expectedSucceeded;
+
+  static SinglePhaseParallelIndexingProgress notRunning()
+  {
+    return new SinglePhaseParallelIndexingProgress(0, 0, 0, 0, 0, -1);
+  }
+
+  @JsonCreator
+  SinglePhaseParallelIndexingProgress(
+      @JsonProperty("running") int running,
+      @JsonProperty("succeeded") int succeeded,
+      @JsonProperty("failed") int failed,
+      @JsonProperty("complete") int complete,
+      @JsonProperty("total") int total,
+      @JsonProperty("expectedSucceeded") int expectedSucceeded
+  )
+  {
+    this.running = running;
+    this.succeeded = succeeded;
+    this.failed = failed;
+    this.complete = complete;
+    this.total = total;
+    this.expectedSucceeded = expectedSucceeded;
+  }
+
+  @JsonProperty
+  public int getRunning()
+  {
+    return running;
+  }
+
+  @JsonProperty
+  public int getSucceeded()
+  {
+    return succeeded;
+  }
+
+  @JsonProperty
+  public int getFailed()
+  {
+    return failed;
+  }
+
+  @JsonProperty
+  public int getComplete()
+  {
+    return complete;
+  }
+
+  @JsonProperty
+  public int getTotal()
+  {
+    return total;
+  }
+
+  @JsonProperty
+  public int getExpectedSucceeded()
+  {
+    return expectedSucceeded;
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SubTaskSpec.java
new file mode 100644
index 0000000..aca5164
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SubTaskSpec.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.data.input.InputSplit;
+import io.druid.indexing.common.task.Task;
+
+import java.util.Map;
+
+public abstract class SubTaskSpec<T extends Task>
+{
+  private final String id;
+  private final String groupId;
+  private final String supervisorTaskId;
+  private final Map<String, Object> context;
+  private final InputSplit inputSplit;
+
+  @JsonCreator
+  public SubTaskSpec(
+      String id,
+      String groupId,
+      String supervisorTaskId,
+      Map<String, Object> context,
+      InputSplit inputSplit
+  )
+  {
+    this.id = id;
+    this.groupId = groupId;
+    this.supervisorTaskId = supervisorTaskId;
+    this.context = context;
+    this.inputSplit = inputSplit;
+  }
+
+  @JsonProperty
+  public String getId()
+  {
+    return id;
+  }
+
+  @JsonProperty
+  public String getGroupId()
+  {
+    return groupId;
+  }
+
+  @JsonProperty
+  public String getSupervisorTaskId()
+  {
+    return supervisorTaskId;
+  }
+
+  @JsonProperty
+  public Map<String, Object> getContext()
+  {
+    return context;
+  }
+
+  @JsonProperty
+  public InputSplit getInputSplit()
+  {
+    return inputSplit;
+  }
+
+  public abstract T newSubTask(int numAttempts);
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskHistory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskHistory.java
new file mode 100644
index 0000000..cb42315
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskHistory.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.task.Task;
+
+import java.util.List;
+
+/**
+ * Task attempt history for complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if its last status is
+ * succeeded or failed.
+ */
+class TaskHistory<T extends Task>
+{
+  private final SubTaskSpec<T> spec;
+  private final List<TaskStatusPlus> attemptHistory; // old attempts to recent ones
+
+  TaskHistory(SubTaskSpec<T> spec, List<TaskStatusPlus> attemptHistory)
+  {
+    attemptHistory.forEach(status -> {
+      Preconditions.checkState(
+          status.getState() == TaskState.SUCCESS || status.getState() == TaskState.FAILED,
+          "Complete tasks should be recorded, but the state of task[%s] is [%s]",
+          status.getId(),
+          status.getState()
+      );
+    });
+    this.spec = spec;
+    this.attemptHistory = ImmutableList.copyOf(attemptHistory);
+  }
+
+  SubTaskSpec<T> getSpec()
+  {
+    return spec;
+  }
+
+  List<TaskStatusPlus> getAttemptHistory()
+  {
+    return attemptHistory;
+  }
+
+  boolean isEmpty()
+  {
+    return attemptHistory.isEmpty();
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskMonitor.java
new file mode 100644
index 0000000..f0b2e3b
--- /dev/null
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskMonitor.java
@@ -0,0 +1,477 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.client.indexing.TaskStatusResponse;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.task.Task;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
+import io.druid.java.util.common.logger.Logger;
+
+import javax.annotation.Nullable;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Responsible for submitting tasks, monitoring task statuses, resubmitting failed tasks, and returning the final task
+ * status.
+ */
+public class TaskMonitor<T extends Task>
+{
+  private static final Logger log = new Logger(TaskMonitor.class);
+
+  private final ScheduledExecutorService taskStatusChecker = Execs.scheduledSingleThreaded(("task-monitor-%d"));
+
+  /**
+   * A map of subTaskSpecId to {@link MonitorEntry}. This map stores the state of running {@link SubTaskSpec}s. This is
+   * read in {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and updated in {@link #submit}
+   * and {@link #retry}. This can also be read by calling {@link #getRunningTaskMonitorEntry},
+   * {@link #getRunningTaskIds}, and {@link #getRunningSubTaskSpecs}.
+   */
+  private final ConcurrentMap<String, MonitorEntry> runningTasks = new ConcurrentHashMap<>();
+
+  /**
+   * A map of subTaskSpecId to {@link TaskHistory}. This map stores the history of complete {@link SubTaskSpec}s
+   * whether their final state is succeeded or failed. This is updated in {@link MonitorEntry#setLastStatus} which is
+   * called by the {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and can be
+   * read by outside of this class.
+   */
+  private final ConcurrentMap<String, TaskHistory<T>> taskHistories = new ConcurrentHashMap<>();
+
+  // lock for updating numRunningTasks, numSucceededTasks, and numFailedTasks
+  private final Object taskCountLock = new Object();
+
+  // lock for updating running state
+  private final Object startStopLock = new Object();
+
+  // overlord client
+  private final IndexingServiceClient indexingServiceClient;
+  private final int maxRetry;
+  private final int expectedNumSucceededTasks;
+
+  private int numRunningTasks;
+  private int numSucceededTasks;
+  private int numFailedTasks;
+
+  private boolean running = false;
+
+  TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry, int expectedNumSucceededTasks)
+  {
+    this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient");
+    this.maxRetry = maxRetry;
+    this.expectedNumSucceededTasks = expectedNumSucceededTasks;
+
+    log.info("TaskMonitor is initialized with expectedNumSucceededTasks[%d]", expectedNumSucceededTasks);
+  }
+
+  public void start(long taskStatusCheckingPeriod)
+  {
+    synchronized (startStopLock) {
+      running = true;
+      log.info("Starting taskMonitor");
+      // NOTE: This polling can be improved to event-driven pushing by registering TaskRunnerListener to TaskRunner.
+      // That listener should be able to send the events reported to TaskRunner to this TaskMonitor.
+      taskStatusChecker.scheduleAtFixedRate(
+          () -> {
+            try {
+              final Iterator<Entry<String, MonitorEntry>> iterator = runningTasks.entrySet().iterator();
+              while (iterator.hasNext()) {
+                final Entry<String, MonitorEntry> entry = iterator.next();
+                final String specId = entry.getKey();
+                final MonitorEntry monitorEntry = entry.getValue();
+                final String taskId = monitorEntry.runningTask.getId();
+                final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId);
+                final TaskStatusPlus taskStatus = taskStatusResponse.getStatus();
+                if (taskStatus != null) {
+                  switch (Preconditions.checkNotNull(taskStatus.getState(), "taskState")) {
+                    case SUCCESS:
+                      incrementNumSucceededTasks();
+
+                      // Remote the current entry after updating taskHistories to make sure that task history
+                      // exists either runningTasks or taskHistories.
+                      monitorEntry.setLastStatus(taskStatus);
+                      iterator.remove();
+                      break;
+                    case FAILED:
+                      incrementNumFailedTasks();
+
+                      log.warn("task[%s] failed!", taskId);
+                      if (monitorEntry.numTries() < maxRetry) {
+                        log.info(
+                            "We still have chances[%d/%d] to complete for spec[%s].",
+                            monitorEntry.numTries(),
+                            maxRetry,
+                            monitorEntry.spec.getId()
+                        );
+                        retry(specId, monitorEntry, taskStatus);
+                      } else {
+                        log.error(
+                            "spec[%s] failed after [%d] tries",
+                            monitorEntry.spec.getId(),
+                            monitorEntry.numTries()
+                        );
+                        // Remote the current entry after updating taskHistories to make sure that task history
+                        // exists either runningTasks or taskHistories.
+                        monitorEntry.setLastStatus(taskStatus);
+                        iterator.remove();
+                      }
+                      break;
+                    case RUNNING:
+                      monitorEntry.updateStatus(taskStatus);
+                      break;
+                    default:
+                      throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getState(), taskId);
+                  }
+                }
+              }
+            }
+            catch (Throwable t) {
+              log.error(t, "Error while monitoring");
+              throw t;
+            }
+          },
+          taskStatusCheckingPeriod,
+          taskStatusCheckingPeriod,
+          TimeUnit.MILLISECONDS
+      );
+    }
+  }
+
+  public void stop()
+  {
+    synchronized (startStopLock) {
+      running = false;
+      taskStatusChecker.shutdownNow();
+      log.info("Stopped taskMonitor");
+    }
+  }
+
+  public ListenableFuture<SubTaskCompleteEvent<T>> submit(SubTaskSpec<T> spec)
+  {
+    synchronized (startStopLock) {
+      if (!running) {
+        return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running"));
+      }
+      final T task = spec.newSubTask(0);
+      log.info("Submitting a new task[%s] for spec[%s]", task.getId(), spec.getId());
+      indexingServiceClient.runTask(task);
+      incrementNumRunningTasks();
+
+      final SettableFuture<SubTaskCompleteEvent<T>> taskFuture = SettableFuture.create();
+      runningTasks.put(
+          spec.getId(),
+          new MonitorEntry(spec, task, indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture)
+      );
+
+      return taskFuture;
+    }
+  }
+
+  /**
+   * Submit a retry task for a failed spec. This method should be called inside of the
+   * {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker}.
+   */
+  private void retry(String subTaskSpecId, MonitorEntry monitorEntry, TaskStatusPlus lastFailedTaskStatus)
+  {
+    synchronized (startStopLock) {
+      if (running) {
+        final SubTaskSpec<T> spec = monitorEntry.spec;
+        final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1);
+        log.info("Submitting a new task[%s] for retrying spec[%s]", task.getId(), spec.getId());
+        indexingServiceClient.runTask(task);
+        incrementNumRunningTasks();
+
+        runningTasks.put(
+            subTaskSpecId,
+            monitorEntry.withNewRunningTask(
+                task,
+                indexingServiceClient.getTaskStatus(task.getId()).getStatus(),
+                lastFailedTaskStatus
+            )
+        );
+      }
+    }
+  }
+
+  /**
+   * This method should be called after {@link #stop()} to make sure no additional tasks are submitted.
+   */
+  void killAll()
+  {
+    runningTasks.values().forEach(entry -> {
+      final String taskId = entry.runningTask.getId();
+      log.info("Request to kill subtask[%s]", taskId);
+      indexingServiceClient.killTask(taskId);
+    });
+    runningTasks.clear();
+  }
+
+  void incrementNumRunningTasks()
+  {
+    synchronized (taskCountLock) {
+      numRunningTasks++;
+    }
+  }
+
+  void incrementNumSucceededTasks()
+  {
+    synchronized (taskCountLock) {
+      numRunningTasks--;
+      numSucceededTasks++;
+      log.info("[%d/%d] tasks succeeded", numSucceededTasks, expectedNumSucceededTasks);
+    }
+  }
+
+  void incrementNumFailedTasks()
+  {
+    synchronized (taskCountLock) {
+      numRunningTasks--;
+      numFailedTasks++;
+    }
+  }
+
+  boolean isSucceeded()
+  {
+    synchronized (taskCountLock) {
+      return numSucceededTasks == expectedNumSucceededTasks;
+    }
+  }
+
+  int getNumRunningTasks()
+  {
+    synchronized (taskCountLock) {
+      return numRunningTasks;
+    }
+  }
+
+  SinglePhaseParallelIndexingProgress getProgress()
+  {
+    synchronized (taskCountLock) {
+      return new SinglePhaseParallelIndexingProgress(
+          numRunningTasks,
+          numSucceededTasks,
+          numFailedTasks,
+          numSucceededTasks + numFailedTasks,
+          numRunningTasks + numSucceededTasks + numFailedTasks,
+          expectedNumSucceededTasks
+      );
+    }
+  }
+
+  Set<String> getRunningTaskIds()
+  {
+    return runningTasks.values().stream().map(entry -> entry.runningTask.getId()).collect(Collectors.toSet());
+  }
+
+  List<SubTaskSpec<T>> getRunningSubTaskSpecs()
+  {
+    return runningTasks.values().stream().map(monitorEntry -> monitorEntry.spec).collect(Collectors.toList());
+  }
+
+  @Nullable
+  MonitorEntry getRunningTaskMonitorEntry(String subTaskSpecId)
+  {
+    return runningTasks.values()
+                       .stream()
+                       .filter(monitorEntry -> monitorEntry.spec.getId().equals(subTaskSpecId))
+                       .findFirst()
+                       .orElse(null);
+  }
+
+  List<SubTaskSpec<T>> getCompleteSubTaskSpecs()
+  {
+    return taskHistories.values().stream().map(TaskHistory::getSpec).collect(Collectors.toList());
+  }
+
+  @Nullable
+  TaskHistory<T> getCompleteSubTaskSpecHistory(String subTaskSpecId)
+  {
+    return taskHistories.get(subTaskSpecId);
+  }
+
+  class MonitorEntry
+  {
+    private final SubTaskSpec<T> spec;
+    private final T runningTask;
+    // old tasks to recent tasks. running task is not included
+    private final CopyOnWriteArrayList<TaskStatusPlus> taskHistory;
+    private final SettableFuture<SubTaskCompleteEvent<T>> completeEventFuture;
+
+    /**
+     * This variable is updated inside of the {@link java.util.concurrent.Callable} executed by
+     * {@link #taskStatusChecker}, and can be read by calling {@link #getRunningStatus}.
+     */
+    @Nullable
+    private volatile TaskStatusPlus runningStatus;
+
+    MonitorEntry(
+        SubTaskSpec<T> spec,
+        T runningTask,
+        @Nullable TaskStatusPlus runningStatus,
+        SettableFuture<SubTaskCompleteEvent<T>> completeEventFuture
+    )
+    {
+      this(spec, runningTask, runningStatus, new CopyOnWriteArrayList<>(), completeEventFuture);
+    }
+
+    private MonitorEntry(
+        SubTaskSpec<T> spec,
+        T runningTask,
+        @Nullable TaskStatusPlus runningStatus,
+        CopyOnWriteArrayList<TaskStatusPlus> taskHistory,
+        SettableFuture<SubTaskCompleteEvent<T>> completeEventFuture
+    )
+    {
+      this.spec = spec;
+      this.runningTask = runningTask;
+      this.runningStatus = runningStatus;
+      this.taskHistory = taskHistory;
+      this.completeEventFuture = completeEventFuture;
+    }
+
+    MonitorEntry withNewRunningTask(T newTask, @Nullable TaskStatusPlus newStatus, TaskStatusPlus statusOfLastTask)
+    {
+      taskHistory.add(statusOfLastTask);
+      return new MonitorEntry(
+          spec,
+          newTask,
+          newStatus,
+          taskHistory,
+          completeEventFuture
+      );
+    }
+
+    int numTries()
+    {
+      return taskHistory.size() + 1; // count runningTask as well
+    }
+
+    void updateStatus(TaskStatusPlus statusPlus)
+    {
+      if (!runningTask.getId().equals(statusPlus.getId())) {
+        throw new ISE(
+            "Task id[%s] of lastStatus is different from the running task[%s]",
+            statusPlus.getId(),
+            runningTask.getId()
+        );
+      }
+      this.runningStatus = statusPlus;
+    }
+
+    void setLastStatus(TaskStatusPlus lastStatus)
+    {
+      if (!runningTask.getId().equals(lastStatus.getId())) {
+        throw new ISE(
+            "Task id[%s] of lastStatus is different from the running task[%s]",
+            lastStatus.getId(),
+            runningTask.getId()
+        );
+      }
+
+      this.runningStatus = lastStatus;
+      taskHistory.add(lastStatus);
+      taskHistories.put(spec.getId(), new TaskHistory<>(spec, taskHistory));
+      completeEventFuture.set(SubTaskCompleteEvent.success(spec, lastStatus));
+    }
+
+    SubTaskSpec<T> getSpec()
+    {
+      return spec;
+    }
+
+    @Nullable
+    TaskStatusPlus getRunningStatus()
+    {
+      return runningStatus;
+    }
+
+    List<TaskStatusPlus> getTaskHistory()
+    {
+      return taskHistory;
+    }
+  }
+
+  static class SubTaskCompleteEvent<T extends Task>
+  {
+    private final SubTaskSpec<T> spec;
+    @Nullable
+    private final TaskStatusPlus lastStatus;
+    @Nullable
+    private final Throwable throwable;
+
+    static <T extends Task> SubTaskCompleteEvent<T> success(SubTaskSpec<T> spec, TaskStatusPlus lastStatus)
+    {
+      return new SubTaskCompleteEvent<>(spec, Preconditions.checkNotNull(lastStatus, "lastStatus"), null);
+    }
+
+    static <T extends Task> SubTaskCompleteEvent<T> fail(SubTaskSpec<T> spec, Throwable t)
+    {
+      return new SubTaskCompleteEvent<>(spec, null, t);
+    }
+
+    private SubTaskCompleteEvent(
+        SubTaskSpec<T> spec,
+        @Nullable TaskStatusPlus lastStatus,
+        @Nullable Throwable throwable
+    )
+    {
+      this.spec = Preconditions.checkNotNull(spec, "spec");
+      this.lastStatus = lastStatus;
+      this.throwable = throwable;
+    }
+
+    SubTaskSpec<T> getSpec()
+    {
+      return spec;
+    }
+
+    TaskState getLastState()
+    {
+      return lastStatus == null ? TaskState.FAILED : lastStatus.getState();
+    }
+
+    @Nullable
+    TaskStatusPlus getLastStatus()
+    {
+      return lastStatus;
+    }
+
+    @Nullable
+    Throwable getThrowable()
+    {
+      return throwable;
+    }
+  }
+}
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java
index c31173e..6326fc0 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java
@@ -42,6 +42,7 @@ import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
 import io.druid.guice.annotations.Self;
+import io.druid.indexer.RunnerTaskState;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexing.common.config.TaskConfig;
@@ -68,6 +69,7 @@ import org.apache.commons.io.FileUtils;
 import org.joda.time.DateTime;
 import org.joda.time.Interval;
 
+import javax.annotation.Nullable;
 import javax.annotation.concurrent.GuardedBy;
 import java.io.File;
 import java.io.IOException;
@@ -666,6 +668,24 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
     }
   }
 
+  @Nullable
+  @Override
+  public RunnerTaskState getRunnerTaskState(String taskId)
+  {
+    final ForkingTaskRunnerWorkItem workItem = tasks.get(taskId);
+    if (workItem == null) {
+      return null;
+    } else {
+      if (workItem.processHolder == null) {
+        return RunnerTaskState.PENDING;
+      } else if (workItem.processHolder.process.isAlive()) {
+        return RunnerTaskState.RUNNING;
+      } else {
+        return RunnerTaskState.NONE;
+      }
+    }
+  }
+
   @Override
   public Optional<ScalingStats> getScalingStats()
   {
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java
index db3a873..3493ab7 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java
@@ -149,6 +149,32 @@ public class HeapMemoryTaskStorage implements TaskStorage
     }
   }
 
+  @Nullable
+  @Override
+  public TaskInfo<Task, TaskStatus> getTaskInfo(String taskId)
+  {
+    giant.lock();
+
+    try {
+      Preconditions.checkNotNull(taskId, "taskId");
+      final TaskStuff taskStuff = tasks.get(taskId);
+      if (taskStuff != null) {
+        return new TaskInfo<>(
+            taskStuff.getTask().getId(),
+            taskStuff.getCreatedDate(),
+            taskStuff.getStatus(),
+            taskStuff.getDataSource(),
+            taskStuff.getTask()
+        );
+      } else {
+        return null;
+      }
+    }
+    finally {
+      giant.unlock();
+    }
+  }
+
   @Override
   public List<Task> getActiveTasks()
   {
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java
index 28f6bbc..77a4c4a 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java
@@ -180,6 +180,13 @@ public class MetadataTaskStorage implements TaskStorage
     return handler.getStatus(taskId);
   }
 
+  @Nullable
+  @Override
+  public TaskInfo<Task, TaskStatus> getTaskInfo(String taskId)
+  {
+    return handler.getTaskInfo(taskId);
+  }
+
   @Override
   public List<Task> getActiveTasks()
   {
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java
index a4c14f9..e8587f6 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java
@@ -45,6 +45,7 @@ import com.google.common.util.concurrent.SettableFuture;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.curator.CuratorUtils;
 import io.druid.curator.cache.PathChildrenCacheFactory;
+import io.druid.indexer.RunnerTaskState;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexing.common.task.Task;
@@ -87,6 +88,7 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Duration;
 import org.joda.time.Period;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.MalformedURLException;
@@ -465,6 +467,23 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
     return ImmutableList.copyOf(Iterables.concat(pendingTasks.values(), runningTasks.values(), completeTasks.values()));
   }
 
+  @Nullable
+  @Override
+  public RunnerTaskState getRunnerTaskState(String taskId)
+  {
+    if (pendingTasks.containsKey(taskId)) {
+      return RunnerTaskState.PENDING;
+    }
+    if (runningTasks.containsKey(taskId)) {
+      return RunnerTaskState.RUNNING;
+    }
+    if (completeTasks.containsKey(taskId)) {
+      return RunnerTaskState.NONE;
+    }
+
+    return null;
+  }
+
   @Override
   public Optional<ScalingStats> getScalingStats()
   {
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java
index c3cb428..3ed6cfd 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java
@@ -994,8 +994,20 @@ public class TaskLockbox
 
     boolean addTask(Task task)
     {
-      Preconditions.checkArgument(taskLock.getGroupId().equals(task.getGroupId()));
-      Preconditions.checkArgument(taskLock.getNonNullPriority() == task.getPriority());
+      Preconditions.checkArgument(
+          taskLock.getGroupId().equals(task.getGroupId()),
+          "groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]",
+          task.getGroupId(),
+          task.getId(),
+          taskLock.getGroupId()
+      );
+      Preconditions.checkArgument(
+          taskLock.getNonNullPriority() == task.getPriority(),
+          "priority[%s] of task[%s] is different from the existing lockPosse's priority[%s]",
+          task.getPriority(),
+          task.getId(),
+          taskLock.getNonNullPriority()
+      );
       return taskIds.add(task.getId());
     }
 
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java
index 1ead0ab..3a63f06 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java
@@ -22,11 +22,13 @@ package io.druid.indexing.overlord;
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.ListenableFuture;
 import io.druid.guice.annotations.PublicApi;
+import io.druid.indexer.RunnerTaskState;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexing.common.task.Task;
 import io.druid.indexing.overlord.autoscaling.ScalingStats;
 import io.druid.java.util.common.Pair;
 
+import javax.annotation.Nullable;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.Executor;
@@ -94,6 +96,12 @@ public interface TaskRunner
 
   Collection<? extends TaskRunnerWorkItem> getKnownTasks();
 
+  @Nullable
+  default RunnerTaskState getRunnerTaskState(String taskId)
+  {
+    return null;
+  }
+
   /**
    * Some runners are able to scale up and down their capacity in a dynamic manner. This returns stats on those activities
    *
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java
index 89849e5..521fe38 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java
@@ -95,6 +95,9 @@ public interface TaskStorage
    */
   Optional<TaskStatus> getStatus(String taskid);
 
+  @Nullable
+  TaskInfo<Task, TaskStatus> getTaskInfo(String taskId);
+
   /**
    * Add an action taken by a task to the audit log.
    *
@@ -122,8 +125,8 @@ public interface TaskStorage
   List<Task> getActiveTasks();
 
   /**
-   * Returns a list of currently running or pending tasks as stored in the storage facility as {@link TaskInfo}. No particular order
-   * is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
+   * Returns a list of currently running or pending tasks as stored in the storage facility as {@link TaskInfo}. No
+   * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
    *
    * @param dataSource datasource
    *
@@ -132,9 +135,10 @@ public interface TaskStorage
   List<TaskInfo<Task, TaskStatus>> getActiveTaskInfo(@Nullable String dataSource);
 
   /**
-   * Returns up to {@code maxTaskStatuses} {@link TaskInfo} objects of recently finished tasks as stored in the storage facility. No
-   * particular order is guaranteed, but implementations are encouraged to return tasks in descending order of creation.
-   * No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply return nothing.
+   * Returns up to {@code maxTaskStatuses} {@link TaskInfo} objects of recently finished tasks as stored in the storage
+   * facility. No particular order is guaranteed, but implementations are encouraged to return tasks in descending order
+   * of creation. No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply
+   * return nothing.
    *
    * @param maxTaskStatuses maxTaskStatuses
    * @param duration        duration
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java
index da7dd71..d67d4ea 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java
@@ -77,6 +77,12 @@ public class TaskStorageQueryAdapter
     return storage.getStatus(taskid);
   }
 
+  @Nullable
+  public TaskInfo<Task, TaskStatus> getTaskInfo(String taskId)
+  {
+    return storage.getTaskInfo(taskId);
+  }
+
   /**
    * Returns all segments created by this task.
    *
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
index 0e72e9d..36fc44a 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
@@ -41,6 +41,7 @@ import io.druid.discovery.DiscoveryDruidNode;
 import io.druid.discovery.DruidNodeDiscovery;
 import io.druid.discovery.DruidNodeDiscoveryProvider;
 import io.druid.discovery.WorkerNodeService;
+import io.druid.indexer.RunnerTaskState;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexing.common.task.Task;
@@ -1120,6 +1121,27 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
                 .collect(Collectors.toList());
   }
 
+  @Nullable
+  @Override
+  public RunnerTaskState getRunnerTaskState(String taskId)
+  {
+    final HttpRemoteTaskRunnerWorkItem workItem = tasks.get(taskId);
+    if (workItem == null) {
+      return null;
+    } else {
+      switch (workItem.state) {
+        case PENDING:
+          return RunnerTaskState.PENDING;
+        case RUNNING:
+          return RunnerTaskState.RUNNING;
+        case COMPLETE:
+          return RunnerTaskState.NONE;
+        default:
+          throw new ISE("Unknown state[%s]", workItem.state);
+      }
+    }
+  }
+
   public List<String> getBlacklistedWorkers()
   {
     return blackListedWorkers.values().stream().map(
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java
index 8b8d5e7..6eb428c 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java
@@ -101,6 +101,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiFunction;
 import java.util.stream.Collectors;
 
 /**
@@ -243,10 +244,61 @@ public class OverlordResource
   @ResourceFilters(TaskResourceFilter.class)
   public Response getTaskStatus(@PathParam("taskid") String taskid)
   {
-    final TaskStatusResponse response = new TaskStatusResponse(
-        taskid,
-        taskStorageQueryAdapter.getStatus(taskid).orNull()
-    );
+    final TaskInfo<Task, TaskStatus> taskInfo = taskStorageQueryAdapter.getTaskInfo(taskid);
+    TaskStatusResponse response = null;
+
+    if (taskInfo != null) {
+      if (taskMaster.getTaskRunner().isPresent()) {
+        final TaskRunner taskRunner = taskMaster.getTaskRunner().get();
+        final TaskRunnerWorkItem workItem = taskRunner
+            .getKnownTasks()
+            .stream()
+            .filter(item -> item.getTaskId().equals(taskid))
+            .findAny()
+            .orElse(null);
+        if (workItem != null) {
+          response = new TaskStatusResponse(
+              workItem.getTaskId(),
+              new TaskStatusPlus(
+                  taskInfo.getId(),
+                  taskInfo.getTask() == null ? null : taskInfo.getTask().getType(),
+                  taskInfo.getCreatedTime(),
+                  // Would be nice to include the real queue insertion time, but the
+                  // TaskStorage API doesn't yet allow it.
+                  DateTimes.EPOCH,
+                  taskInfo.getStatus().getStatusCode(),
+                  taskRunner.getRunnerTaskState(workItem.getTaskId()),
+                  taskInfo.getStatus().getDuration(),
+                  workItem.getLocation(),
+                  taskInfo.getDataSource(),
+                  taskInfo.getStatus().getErrorMsg()
+              )
+          );
+        }
+      }
+
+      if (response == null) {
+        response = new TaskStatusResponse(
+            taskid,
+            new TaskStatusPlus(
+                taskInfo.getId(),
+                taskInfo.getTask() == null ? null : taskInfo.getTask().getType(),
+                taskInfo.getCreatedTime(),
+                // Would be nice to include the real queue insertion time, but the
+                // TaskStorage API doesn't yet allow it.
+                DateTimes.EPOCH,
+                taskInfo.getStatus().getStatusCode(),
+                RunnerTaskState.WAITING,
+                taskInfo.getStatus().getDuration(),
+                TaskLocation.unknown(),
+                taskInfo.getDataSource(),
+                taskInfo.getStatus().getErrorMsg()
+            )
+        );
+      }
+    } else {
+      response = new TaskStatusResponse(taskid, null);
+    }
 
     final Response.Status status = response.getStatus() == null
                                    ? Response.Status.NOT_FOUND
@@ -626,7 +678,9 @@ public class OverlordResource
       final List<TaskInfo<Task, TaskStatus>> taskInfoList = taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(
           maxCompletedTasks, duration, dataSource
       );
-      final List<TaskStatusPlus> completedTasks = Lists.transform(taskInfoList, completeTaskTransformFunc);
+      final List<TaskStatusPlus> completedTasks = taskInfoList.stream()
+                                                              .map(completeTaskTransformFunc::apply)
+                                                              .collect(Collectors.toList());
       finalTaskList.addAll(completedTasks);
     }
 
@@ -651,17 +705,23 @@ public class OverlordResource
     }
     if (state == null || "waiting".equals(StringUtils.toLowerCase(state))) {
       final List<AnyTask> waitingWorkItems = filterActiveTasks(RunnerTaskState.WAITING, allActiveTasks);
-      List<TaskStatusPlus> transformedWaitingList = Lists.transform(waitingWorkItems, activeTaskTransformFunc);
+      List<TaskStatusPlus> transformedWaitingList = waitingWorkItems.stream()
+                                                                    .map(activeTaskTransformFunc::apply)
+                                                                    .collect(Collectors.toList());
       finalTaskList.addAll(transformedWaitingList);
     }
     if (state == null || "pending".equals(StringUtils.toLowerCase(state))) {
       final List<AnyTask> pendingWorkItems = filterActiveTasks(RunnerTaskState.PENDING, allActiveTasks);
-      List<TaskStatusPlus> transformedPendingList = Lists.transform(pendingWorkItems, activeTaskTransformFunc);
+      List<TaskStatusPlus> transformedPendingList = pendingWorkItems.stream()
+                                                                    .map(activeTaskTransformFunc::apply)
+                                                                    .collect(Collectors.toList());
       finalTaskList.addAll(transformedPendingList);
     }
     if (state == null || "running".equals(StringUtils.toLowerCase(state))) {
       final List<AnyTask> runningWorkItems = filterActiveTasks(RunnerTaskState.RUNNING, allActiveTasks);
-      List<TaskStatusPlus> transformedRunningList = Lists.transform(runningWorkItems, activeTaskTransformFunc);
+      List<TaskStatusPlus> transformedRunningList = runningWorkItems.stream()
+                                                                    .map(activeTaskTransformFunc::apply)
+                                                                    .collect(Collectors.toList());
       finalTaskList.addAll(transformedRunningList);
     }
     final List<TaskStatusPlus> authorizedList = securedTaskStatusPlus(
@@ -673,6 +733,24 @@ public class OverlordResource
     return Response.ok(authorizedList).build();
   }
 
+  private static BiFunction<TaskInfo<Task, TaskStatus>, RunnerTaskState, TaskStatusPlus> newTaskInfo2TaskStatusPlusFn()
+  {
+    return (taskInfo, runnerTaskState) -> new TaskStatusPlus(
+        taskInfo.getId(),
+        taskInfo.getTask() == null ? null : taskInfo.getTask().getType(),
+        taskInfo.getCreatedTime(),
+        // Would be nice to include the real queue insertion time, but the
+        // TaskStorage API doesn't yet allow it.
+        DateTimes.EPOCH,
+        taskInfo.getStatus().getStatusCode(),
+        runnerTaskState,
+        taskInfo.getStatus().getDuration(),
+        TaskLocation.unknown(),
+        taskInfo.getDataSource(),
+        taskInfo.getStatus().getErrorMsg()
+    );
+  }
+
   private List<AnyTask> filterActiveTasks(
       RunnerTaskState state,
       List<AnyTask> allTasks
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java
index 9883ee9..6ce9ef6 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java
@@ -21,19 +21,21 @@ package io.druid.indexing.overlord.http;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import io.druid.indexer.TaskStatus;
+import io.druid.indexer.TaskStatusPlus;
 
+import javax.annotation.Nullable;
 import java.util.Objects;
 
 public class TaskStatusResponse
 {
   private final String task; // Task ID, named "task" in the JSONification of this class.
-  private final TaskStatus status;
+  @Nullable
+  private final TaskStatusPlus status;
 
   @JsonCreator
   public TaskStatusResponse(
       @JsonProperty("task") final String task,
-      @JsonProperty("status") final TaskStatus status
+      @JsonProperty("status") @Nullable final TaskStatusPlus status
   )
   {
     this.task = task;
@@ -47,7 +49,8 @@ public class TaskStatusResponse
   }
 
   @JsonProperty
-  public TaskStatus getStatus()
+  @Nullable
+  public TaskStatusPlus getStatus()
   {
     return status;
   }
@@ -69,7 +72,6 @@ public class TaskStatusResponse
   @Override
   public int hashCode()
   {
-
     return Objects.hash(task, status);
   }
 
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java
index c56f5c2..975fcb9 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java
@@ -23,6 +23,9 @@ import com.fasterxml.jackson.databind.InjectableValues;
 import com.fasterxml.jackson.databind.Module;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableMap;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.client.indexing.NoopIndexingServiceClient;
 import io.druid.guice.ServerModule;
 import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
 import io.druid.indexing.common.stats.RowIngestionMetersFactory;
@@ -81,7 +84,7 @@ public class TestUtils
 
     jsonMapper.setInjectableValues(
         new InjectableValues.Std()
-            .addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE)
+            .addValue(ExprMacroTable.class, LookupEnabledTestExprMacroTable.INSTANCE)
             .addValue(IndexIO.class, indexIO)
             .addValue(ObjectMapper.class, jsonMapper)
             .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider())
@@ -89,6 +92,8 @@ public class TestUtils
             .addValue(AuthorizerMapper.class, null)
             .addValue(RowIngestionMetersFactory.class, rowIngestionMetersFactory)
             .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT)
+            .addValue(IndexingServiceClient.class, new NoopIndexingServiceClient())
+            .addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of()))
     );
   }
 
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java
index f9d7079..8309d95 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java
@@ -127,7 +127,7 @@ public class SegmentInsertActionTest
     actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
 
     thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task"));
+    thrown.expectMessage(CoreMatchers.containsString("are not covered by locks"));
     final Set<DataSegment> segments = actionTestKit.getTaskLockbox().doInCriticalSection(
         task,
         Collections.singletonList(INTERVAL),
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java
index a8acbe8..c0a79df 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java
@@ -155,7 +155,7 @@ public class SegmentTransactionalInsertActionTest
     actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
 
     thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task"));
+    thrown.expectMessage(CoreMatchers.containsString("are not covered by locks"));
     SegmentPublishResult result = action.perform(task, actionTestKit.getTaskActionToolbox());
     Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(SEGMENT3), true), result);
   }
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java
new file mode 100644
index 0000000..b21a82b
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.actions;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.indexing.common.TaskLock;
+import io.druid.indexing.common.TaskLockType;
+import io.druid.jackson.DefaultObjectMapper;
+import io.druid.java.util.common.Intervals;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class SurrogateActionTest
+{
+  @Test
+  public void testSerde() throws IOException
+  {
+    final ObjectMapper objectMapper = new DefaultObjectMapper();
+    final SurrogateAction<TaskLock, LockTryAcquireAction> surrogateAction = new SurrogateAction<>(
+        "testId", new LockTryAcquireAction(TaskLockType.EXCLUSIVE, Intervals.of("2018-01-01/2019-01-01"))
+    );
+
+    final String json = objectMapper.writeValueAsString(surrogateAction);
+    Assert.assertEquals(surrogateAction.toString(), objectMapper.readValue(json, TaskAction.class).toString());
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java
index 5212d5b..3194863 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java
@@ -21,6 +21,7 @@ package io.druid.indexing.common.actions;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Suppliers;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.common.TestUtils;
 import io.druid.indexing.common.config.TaskStorageConfig;
 import io.druid.indexing.overlord.HeapMemoryTaskStorage;
@@ -94,9 +95,11 @@ public class TaskActionTestKit extends ExternalResource
     );
     taskActionToolbox = new TaskActionToolbox(
         taskLockbox,
+        taskStorage,
         metadataStorageCoordinator,
         new NoopServiceEmitter(),
-        EasyMock.createMock(SupervisorManager.class)
+        EasyMock.createMock(SupervisorManager.class),
+        new Counters()
     );
     testDerbyConnector.createDataSourceTable();
     testDerbyConnector.createPendingSegmentsTable();
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
index 6d3b316..9f4073b 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
@@ -48,6 +48,7 @@ import io.druid.discovery.DruidNodeAnnouncer;
 import io.druid.discovery.LookupNodeService;
 import io.druid.indexer.IngestionState;
 import io.druid.indexer.TaskState;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
 import io.druid.indexing.common.SegmentLoaderFactory;
 import io.druid.indexing.common.TaskReport;
@@ -1393,9 +1394,11 @@ public class AppenderatorDriverRealtimeIndexTaskTest
 
     final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
         taskLockbox,
+        taskStorage,
         mdc,
         emitter,
-        EasyMock.createMock(SupervisorManager.class)
+        EasyMock.createMock(SupervisorManager.class),
+        new Counters()
     );
     final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
         taskStorage,
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java
new file mode 100644
index 0000000..c9071fc
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexing.common.Counters;
+import io.druid.indexing.common.TestUtils;
+import io.druid.indexing.common.actions.LocalTaskActionClient;
+import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.actions.TaskActionToolbox;
+import io.druid.indexing.common.config.TaskStorageConfig;
+import io.druid.indexing.overlord.HeapMemoryTaskStorage;
+import io.druid.indexing.overlord.TaskLockbox;
+import io.druid.indexing.overlord.TaskStorage;
+import io.druid.metadata.EntryExistsException;
+import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
+import io.druid.metadata.TestDerbyConnector;
+import io.druid.segment.IndexIO;
+import io.druid.segment.IndexMergerV9;
+import io.druid.server.metrics.NoopServiceEmitter;
+import org.junit.Rule;
+
+public abstract class IngestionTestBase
+{
+  @Rule
+  public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule();
+
+  private final TestUtils testUtils = new TestUtils();
+  private final ObjectMapper objectMapper = testUtils.getTestObjectMapper();
+  private final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null));
+  private final TaskLockbox lockbox = new TaskLockbox(taskStorage);
+
+  public IngestionTestBase()
+  {
+  }
+
+  public TaskActionClient createActionClient(Task task)
+  {
+    return new LocalTaskActionClient(task, taskStorage, createTaskActionToolbox());
+  }
+
+  public void prepareTaskForLocking(Task task) throws EntryExistsException
+  {
+    lockbox.add(task);
+    taskStorage.insert(task, TaskStatus.running(task.getId()));
+  }
+
+  public ObjectMapper getObjectMapper()
+  {
+    return objectMapper;
+  }
+
+  public TaskStorage getTaskStorage()
+  {
+    return taskStorage;
+  }
+
+  public TaskLockbox getLockbox()
+  {
+    return lockbox;
+  }
+
+  public TaskActionToolbox createTaskActionToolbox()
+  {
+    final IndexerSQLMetadataStorageCoordinator storageCoordinator = new IndexerSQLMetadataStorageCoordinator(
+        objectMapper,
+        derbyConnectorRule.metadataTablesConfigSupplier().get(),
+        derbyConnectorRule.getConnector()
+    );
+    storageCoordinator.start();
+    return new TaskActionToolbox(
+        lockbox,
+        taskStorage,
+        storageCoordinator,
+        new NoopServiceEmitter(),
+        null,
+        new Counters()
+    );
+  }
+
+  public IndexIO getIndexIO()
+  {
+    return testUtils.getTestIndexIO();
+  }
+
+  public IndexMergerV9 getIndexMerger()
+  {
+    return testUtils.getTestIndexMergerV9();
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
index ac523e5..d3edfef 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -45,6 +45,7 @@ import io.druid.discovery.DataNodeService;
 import io.druid.discovery.DruidNodeAnnouncer;
 import io.druid.discovery.LookupNodeService;
 import io.druid.indexer.TaskState;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.common.SegmentLoaderFactory;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexing.common.TaskToolbox;
@@ -976,9 +977,11 @@ public class RealtimeIndexTaskTest
     taskLockbox.syncFromStorage();
     final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
         taskLockbox,
+        taskStorage,
         mdc,
         emitter,
-        EasyMock.createMock(SupervisorManager.class)
+        EasyMock.createMock(SupervisorManager.class),
+        new Counters()
     );
     final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
         taskStorage,
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
new file mode 100644
index 0000000..341d5ee
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.client.indexing.NoopIndexingServiceClient;
+import io.druid.client.indexing.TaskStatusResponse;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.CSVParseSpec;
+import io.druid.data.input.impl.DimensionsSpec;
+import io.druid.data.input.impl.ParseSpec;
+import io.druid.data.input.impl.TimestampSpec;
+import io.druid.indexer.RunnerTaskState;
+import io.druid.indexer.TaskLocation;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.TaskInfoProvider;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
+import io.druid.indexing.common.task.IndexTaskClientFactory;
+import io.druid.indexing.common.task.IngestionTestBase;
+import io.druid.indexing.common.task.NoopTestTaskFileWriter;
+import io.druid.indexing.common.task.Task;
+import io.druid.indexing.common.task.TaskResource;
+import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
+import io.druid.segment.loading.DataSegmentKiller;
+import io.druid.segment.loading.LocalDataSegmentPusher;
+import io.druid.segment.loading.LocalDataSegmentPusherConfig;
+import io.druid.segment.realtime.appenderator.SegmentIdentifier;
+import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
+import io.druid.server.security.AllowAllAuthorizer;
+import io.druid.server.security.Authorizer;
+import io.druid.server.security.AuthorizerMapper;
+import io.druid.timeline.DataSegment;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.stream.Stream;
+
+public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
+{
+  static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec(
+      new TimestampSpec(
+          "ts",
+          "auto",
+          null
+      ),
+      new DimensionsSpec(
+          DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")),
+          Lists.newArrayList(),
+          Lists.newArrayList()
+      ),
+      null,
+      Arrays.asList("ts", "dim", "val"),
+      false,
+      0
+  );
+
+  TaskActionClient actionClient;
+  LocalIndexingServiceClient indexingServiceClient;
+  TaskToolbox toolbox;
+  File localDeepStorage;
+
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  class LocalIndexingServiceClient extends NoopIndexingServiceClient
+  {
+    private final ConcurrentMap<String, Future<TaskStatus>> tasks = new ConcurrentHashMap<>();
+    private final ListeningExecutorService service = MoreExecutors.listeningDecorator(
+        Execs.multiThreaded(5, "parallel-index-supervisor-task-test-%d")
+    );
+
+    @Override
+    public String runTask(Object taskObject)
+    {
+      final ParallelIndexSubTask subTask = (ParallelIndexSubTask) taskObject;
+      tasks.put(subTask.getId(), service.submit(() -> {
+        try {
+          final TaskToolbox toolbox = createTaskToolbox(subTask);
+          if (subTask.isReady(toolbox.getTaskActionClient())) {
+            return subTask.run(toolbox);
+          } else {
+            throw new ISE("task[%s] is not ready", subTask.getId());
+          }
+        }
+        catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }));
+      return subTask.getId();
+    }
+
+    @Override
+    public TaskStatusResponse getTaskStatus(String taskId)
+    {
+      final Future<TaskStatus> taskStatusFuture = tasks.get(taskId);
+      if (taskStatusFuture != null) {
+        try {
+          if (taskStatusFuture.isDone()) {
+            final TaskStatus taskStatus = taskStatusFuture.get();
+            return new TaskStatusResponse(
+                taskId,
+                new TaskStatusPlus(
+                    taskId,
+                    "index_sub",
+                    DateTimes.EPOCH,
+                    DateTimes.EPOCH,
+                    taskStatus.getStatusCode(),
+                    RunnerTaskState.NONE,
+                    -1L,
+                    TaskLocation.unknown(),
+                    null,
+                    null
+                )
+            );
+          } else {
+            return new TaskStatusResponse(
+                taskId,
+                new TaskStatusPlus(
+                    taskId,
+                    "index_sub",
+                    DateTimes.EPOCH,
+                    DateTimes.EPOCH,
+                    TaskState.RUNNING,
+                    RunnerTaskState.RUNNING,
+                    -1L,
+                    TaskLocation.unknown(),
+                    null,
+                    null
+                )
+            );
+          }
+        }
+        catch (InterruptedException | ExecutionException e) {
+          // We don't have a way to pass this exception to the supervisorTask yet..
+          // So, let's print it here.
+          System.err.println(Throwables.getStackTraceAsString(e));
+          return new TaskStatusResponse(
+              taskId,
+              new TaskStatusPlus(
+                  taskId,
+                  "index_sub",
+                  DateTimes.EPOCH,
+                  DateTimes.EPOCH,
+                  TaskState.FAILED,
+                  RunnerTaskState.NONE,
+                  -1L,
+                  TaskLocation.unknown(),
+                  null,
+                  null
+              )
+          );
+        }
+      } else {
+        return new TaskStatusResponse(taskId, null);
+      }
+    }
+
+    @Override
+    public String killTask(String taskId)
+    {
+      final Future<TaskStatus> taskStatusFuture = tasks.remove(taskId);
+      if (taskStatusFuture != null) {
+        taskStatusFuture.cancel(true);
+        return taskId;
+      } else {
+        return null;
+      }
+    }
+
+    void shutdown()
+    {
+      service.shutdownNow();
+    }
+  }
+
+  TaskToolbox createTaskToolbox(Task task) throws IOException
+  {
+    return new TaskToolbox(
+        null,
+        actionClient,
+        null,
+        new LocalDataSegmentPusher(
+            new LocalDataSegmentPusherConfig()
+            {
+              @Override
+              public File getStorageDirectory()
+              {
+                return localDeepStorage;
+              }
+            },
+            getObjectMapper()
+        ),
+        new DataSegmentKiller()
+        {
+          @Override
+          public void kill(DataSegment segment)
+          {
+          }
+
+          @Override
+          public void killAll()
+          {
+          }
+        },
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        getObjectMapper(),
+        temporaryFolder.newFolder(task.getId()),
+        getIndexIO(),
+        null,
+        null,
+        getIndexMerger(),
+        null,
+        null,
+        null,
+        null,
+        new NoopTestTaskFileWriter()
+    );
+  }
+
+  static class TestParallelIndexSupervisorTask extends ParallelIndexSupervisorTask
+  {
+    TestParallelIndexSupervisorTask(
+        String id,
+        TaskResource taskResource,
+        ParallelIndexIngestionSpec ingestionSchema,
+        Map<String, Object> context,
+        IndexingServiceClient indexingServiceClient
+    )
+    {
+      super(
+          id,
+          taskResource,
+          ingestionSchema,
+          context,
+          indexingServiceClient,
+          new NoopChatHandlerProvider(),
+          new AuthorizerMapper(ImmutableMap.of())
+          {
+            @Override
+            public Authorizer getAuthorizer(String name)
+            {
+              return new AllowAllAuthorizer();
+            }
+          },
+          new DropwizardRowIngestionMetersFactory()
+      );
+    }
+
+    @Override
+    public TaskStatus run(TaskToolbox toolbox) throws Exception
+    {
+      return TaskStatus.fromCode(
+          getId(),
+          new TestParallelIndexTaskRunner(
+              toolbox,
+              getId(),
+              getGroupId(),
+              getIngestionSchema(),
+              getContext(),
+              new NoopIndexingServiceClient()
+          ).run()
+      );
+    }
+  }
+
+  static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRunner
+  {
+    TestParallelIndexTaskRunner(
+        TaskToolbox toolbox,
+        String taskId,
+        String groupId,
+        ParallelIndexIngestionSpec ingestionSchema,
+        Map<String, Object> context,
+        @Nullable IndexingServiceClient indexingServiceClient
+    )
+    {
+      super(
+          toolbox,
+          taskId,
+          groupId,
+          ingestionSchema,
+          context,
+          indexingServiceClient
+      );
+    }
+
+    @Override
+    Stream<ParallelIndexSubTaskSpec> subTaskSpecIterator() throws IOException
+    {
+      final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema()
+          .getIOConfig()
+          .getFirehoseFactory();
+      return baseFirehoseFactory.getSplits().map(split -> {
+        try {
+          // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id
+          Thread.sleep(10);
+        }
+        catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+        return newTaskSpec((InputSplit<?>) split);
+      });
+    }
+  }
+
+  static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexTaskClient>
+  {
+    private final ParallelIndexSupervisorTask supervisorTask;
+
+    LocalParallelIndexTaskClientFactory(ParallelIndexSupervisorTask supervisorTask)
+    {
+      this.supervisorTask = supervisorTask;
+    }
+
+    @Override
+    public ParallelIndexTaskClient build(
+        TaskInfoProvider taskInfoProvider,
+        String callerId,
+        int numThreads,
+        Duration httpTimeout,
+        long numRetries
+    )
+    {
+      return new LocalParallelIndexTaskClient(callerId, supervisorTask);
+    }
+  }
+
+  static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient
+  {
+    private final ParallelIndexSupervisorTask supervisorTask;
+
+    LocalParallelIndexTaskClient(String callerId, ParallelIndexSupervisorTask supervisorTask)
+    {
+      super(null, null, null, null, callerId, 0);
+      this.supervisorTask = supervisorTask;
+    }
+
+    @Override
+    public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
+    {
+      return supervisorTask.allocateNewSegment(timestamp);
+    }
+
+    @Override
+    public void report(String supervisorTaskId, List<DataSegment> pushedSegments)
+    {
+      supervisorTask.getRunner().collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments));
+    }
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
new file mode 100644
index 0000000..0891a90
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
@@ -0,0 +1,727 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.StringInputRowParser;
+import io.druid.indexer.RunnerTaskState;
+import io.druid.indexer.TaskLocation;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.TaskLock;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.LockListAction;
+import io.druid.indexing.common.actions.SurrogateAction;
+import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.task.AbstractTask;
+import io.druid.indexing.common.task.IndexTaskClientFactory;
+import io.druid.indexing.common.task.TaskResource;
+import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
+import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.concurrent.Execs;
+import io.druid.java.util.common.granularity.Granularities;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.LongSumAggregatorFactory;
+import io.druid.segment.indexing.DataSchema;
+import io.druid.segment.indexing.granularity.UniformGranularitySpec;
+import io.druid.server.security.AuthConfig;
+import io.druid.server.security.AuthenticationResult;
+import io.druid.timeline.DataSegment;
+import io.druid.timeline.partition.NumberedShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelIndexSupervisorTaskTest
+{
+  private static final int NUM_SUB_TASKS = 10;
+
+  // specId -> spec
+  private final ConcurrentMap<String, ParallelIndexSubTaskSpec> subTaskSpecs = new ConcurrentHashMap<>();
+
+  // specId -> taskStatusPlus
+  private final ConcurrentMap<String, TaskStatusPlus> runningSpecs = new ConcurrentHashMap<>();
+
+  // specId -> taskStatusPlus list
+  private final ConcurrentMap<String, List<TaskStatusPlus>> taskHistories = new ConcurrentHashMap<>();
+
+  // taskId -> subTaskSpec
+  private final ConcurrentMap<String, ParallelIndexSubTaskSpec> taskIdToSpec = new ConcurrentHashMap<>();
+
+  // taskId -> task
+  private final CopyOnWriteArrayList<TestSubTask> runningTasks = new CopyOnWriteArrayList<>();
+
+  private ExecutorService service;
+
+  private TestSupervisorTask task;
+
+  @Before
+  public void setup() throws IOException
+  {
+    service = Execs.singleThreaded("parallel-index-supervisor-task-resource-test-%d");
+    indexingServiceClient = new LocalIndexingServiceClient();
+    localDeepStorage = temporaryFolder.newFolder("localStorage");
+  }
+
+  @After
+  public void teardown()
+  {
+    indexingServiceClient.shutdown();
+    temporaryFolder.delete();
+    service.shutdownNow();
+  }
+
+  @Test(timeout = 20000L)
+  public void testAPIs() throws Exception
+  {
+    task = newTask(
+        Intervals.of("2017/2018"),
+        new ParallelIndexIOConfig(
+            new TestFirehose(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())),
+            false
+        )
+    );
+    actionClient = createActionClient(task);
+    toolbox = createTaskToolbox(task);
+
+    prepareTaskForLocking(task);
+    Assert.assertTrue(task.isReady(actionClient));
+    final Future<TaskStatus> supervisorTaskFuture = service.submit(() -> task.run(toolbox));
+    Thread.sleep(1000);
+
+    final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner();
+    Assert.assertNotNull("runner is null", runner);
+
+    // test getMode
+    Response response = task.getMode(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals("parallel", response.getEntity());
+
+    // test expectedNumSucceededTasks
+    response = task.getProgress(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(NUM_SUB_TASKS, ((SinglePhaseParallelIndexingProgress) response.getEntity()).getExpectedSucceeded());
+
+    // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state.
+    // We use polling to check the state of taskMonitor in this test.
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getRunning) < NUM_SUB_TASKS) {
+      Thread.sleep(100);
+    }
+
+    int succeededTasks = 0;
+    int failedTasks = 0;
+    checkState(
+        succeededTasks,
+        failedTasks,
+        buildStateMap()
+    );
+
+    // numRunningTasks and numSucceededTasks after some successful subTasks
+    succeededTasks += 2;
+    for (int i = 0; i < succeededTasks; i++) {
+      runningTasks.get(0).setState(TaskState.SUCCESS);
+    }
+
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) {
+      Thread.sleep(100);
+    }
+
+    checkState(
+        succeededTasks,
+        failedTasks,
+        buildStateMap()
+    );
+
+    // numRunningTasks and numSucceededTasks after some failed subTasks
+    failedTasks += 3;
+    for (int i = 0; i < failedTasks; i++) {
+      runningTasks.get(0).setState(TaskState.FAILED);
+    }
+
+    // Wait for new tasks to be started
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) {
+      Thread.sleep(100);
+    }
+
+    checkState(
+        succeededTasks,
+        failedTasks,
+        buildStateMap()
+    );
+
+    // Make sure only one subTask is running
+    succeededTasks += 7;
+    for (int i = 0; i < 7; i++) {
+      runningTasks.get(0).setState(TaskState.SUCCESS);
+    }
+
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) {
+      Thread.sleep(100);
+    }
+
+    checkState(
+        succeededTasks,
+        failedTasks,
+        buildStateMap()
+    );
+
+    Assert.assertEquals(1, runningSpecs.size());
+    final String lastRunningSpecId = runningSpecs.keySet().iterator().next();
+    final List<TaskStatusPlus> taskHistory = taskHistories.get(lastRunningSpecId);
+    // This should be a failed task history because new tasks appear later in runningTasks.
+    Assert.assertEquals(1, taskHistory.size());
+
+    // Test one more failure
+    runningTasks.get(0).setState(TaskState.FAILED);
+    failedTasks++;
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks) {
+      Thread.sleep(100);
+    }
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getRunning) < 1) {
+      Thread.sleep(100);
+    }
+
+    checkState(
+        succeededTasks,
+        failedTasks,
+        buildStateMap()
+    );
+    Assert.assertEquals(2, taskHistory.size());
+
+    runningTasks.get(0).setState(TaskState.SUCCESS);
+    succeededTasks++;
+    while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) {
+      Thread.sleep(100);
+    }
+
+    Assert.assertEquals(TaskState.SUCCESS, supervisorTaskFuture.get(1000, TimeUnit.MILLISECONDS).getStatusCode());
+  }
+
+  @SuppressWarnings({"ConstantConditions"})
+  private int getNumSubTasks(Function<SinglePhaseParallelIndexingProgress, Integer> func)
+  {
+    final Response response = task.getProgress(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    return func.apply((SinglePhaseParallelIndexingProgress) response.getEntity());
+  }
+
+  private Map<String, SubTaskSpecStatus> buildStateMap()
+  {
+    final Map<String, SubTaskSpecStatus> stateMap = new HashMap<>();
+    subTaskSpecs.forEach((specId, spec) -> {
+      final List<TaskStatusPlus> taskHistory = taskHistories.get(specId);
+      final TaskStatusPlus runningTaskStatus = runningSpecs.get(specId);
+      stateMap.put(
+          specId,
+          new SubTaskSpecStatus(spec, runningTaskStatus, taskHistory == null ? Collections.emptyList() : taskHistory)
+      );
+    });
+    return stateMap;
+  }
+
+  /**
+   * Test all endpoints of {@link ParallelIndexSupervisorTask}.
+   */
+  private void checkState(
+      int expectedSucceededTasks,
+      int expectedFailedTask,
+      Map<String, SubTaskSpecStatus> expectedSubTaskStateResponses // subTaskSpecId -> response
+  )
+  {
+    Response response = task.getProgress(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    final SinglePhaseParallelIndexingProgress monitorStatus = (SinglePhaseParallelIndexingProgress) response.getEntity();
+
+    // numRunningTasks
+    Assert.assertEquals(runningTasks.size(), monitorStatus.getRunning());
+
+    // numSucceededTasks
+    Assert.assertEquals(expectedSucceededTasks, monitorStatus.getSucceeded());
+
+    // numFailedTasks
+    Assert.assertEquals(expectedFailedTask, monitorStatus.getFailed());
+
+    // numCompleteTasks
+    Assert.assertEquals(expectedSucceededTasks + expectedFailedTask, monitorStatus.getComplete());
+
+    // numTotalTasks
+    Assert.assertEquals(runningTasks.size() + expectedSucceededTasks + expectedFailedTask, monitorStatus.getTotal());
+
+    // runningSubTasks
+    response = task.getRunningTasks(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(
+        runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet()),
+        new HashSet<>((Collection<String>) response.getEntity())
+    );
+
+    // subTaskSpecs
+    response = task.getSubTaskSpecs(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    List<SubTaskSpec<ParallelIndexSubTask>> actualSubTaskSpecMap =
+        (List<SubTaskSpec<ParallelIndexSubTask>>) response.getEntity();
+    Assert.assertEquals(
+        subTaskSpecs.keySet(),
+        actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet())
+    );
+
+    // runningSubTaskSpecs
+    response = task.getRunningSubTaskSpecs(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    actualSubTaskSpecMap =
+        (List<SubTaskSpec<ParallelIndexSubTask>>) response.getEntity();
+    Assert.assertEquals(
+        runningSpecs.keySet(),
+        actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet())
+    );
+
+    // completeSubTaskSpecs
+    final List<SubTaskSpec<ParallelIndexSubTask>> completeSubTaskSpecs = expectedSubTaskStateResponses
+        .entrySet()
+        .stream()
+        .filter(entry -> !runningSpecs.containsKey(entry.getKey()))
+        .map(entry -> entry.getValue().getSpec())
+        .collect(Collectors.toList());
+
+    response = task.getCompleteSubTaskSpecs(newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(completeSubTaskSpecs, response.getEntity());
+
+    // subTaskSpec
+    final String subTaskId = runningSpecs.keySet().iterator().next();
+    response = task.getSubTaskSpec(subTaskId, newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    final SubTaskSpec<ParallelIndexSubTask> subTaskSpec =
+        (SubTaskSpec<ParallelIndexSubTask>) response.getEntity();
+    Assert.assertEquals(subTaskId, subTaskSpec.getId());
+
+    // subTaskState
+    response = task.getSubTaskState(subTaskId, newRequest());
+    Assert.assertEquals(200, response.getStatus());
+    final SubTaskSpecStatus expectedResponse = Preconditions.checkNotNull(
+        expectedSubTaskStateResponses.get(subTaskId),
+        "response for task[%s]",
+        subTaskId
+    );
+    final SubTaskSpecStatus actualResponse = (SubTaskSpecStatus) response.getEntity();
+    Assert.assertEquals(expectedResponse.getSpec().getId(), actualResponse.getSpec().getId());
+    Assert.assertEquals(expectedResponse.getCurrentStatus(), actualResponse.getCurrentStatus());
+    Assert.assertEquals(expectedResponse.getTaskHistory(), actualResponse.getTaskHistory());
+
+    // completeSubTaskSpecAttemptHistory
+    final String completeSubTaskSpecId = expectedSubTaskStateResponses
+        .entrySet()
+        .stream()
+        .filter(entry -> {
+          final TaskStatusPlus currentStatus = entry.getValue().getCurrentStatus();
+          return currentStatus != null &&
+                 (currentStatus.getState() == TaskState.SUCCESS || currentStatus.getState() == TaskState.FAILED);
+        })
+        .map(Entry::getKey)
+        .findFirst()
+        .orElse(null);
+    if (completeSubTaskSpecId != null) {
+      response = task.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest());
+      Assert.assertEquals(200, response.getStatus());
+      Assert.assertEquals(
+          expectedSubTaskStateResponses.get(completeSubTaskSpecId).getTaskHistory(),
+          response.getEntity()
+      );
+    }
+  }
+
+  private static HttpServletRequest newRequest()
+  {
+    final HttpServletRequest request = EasyMock.niceMock(HttpServletRequest.class);
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null);
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT))
+            .andReturn(new AuthenticationResult("test", "test", "test", Collections.emptyMap()));
+    EasyMock.replay(request);
+    return request;
+  }
+
+  private TestSupervisorTask newTask(
+      Interval interval,
+      ParallelIndexIOConfig ioConfig
+  )
+  {
+    // set up ingestion spec
+    final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
+        new DataSchema(
+            "dataSource",
+            getObjectMapper().convertValue(
+                new StringInputRowParser(
+                    DEFAULT_PARSE_SPEC,
+                    null
+                ),
+                Map.class
+            ),
+            new AggregatorFactory[]{
+                new LongSumAggregatorFactory("val", "val")
+            },
+            new UniformGranularitySpec(
+                Granularities.DAY,
+                Granularities.MINUTE,
+                interval == null ? null : Collections.singletonList(interval)
+            ),
+            null,
+            getObjectMapper()
+        ),
+        ioConfig,
+        new ParallelIndexTuningConfig(
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            NUM_SUB_TASKS,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null
+        )
+    );
+
+    // set up test tools
+    return new TestSupervisorTask(
+        null,
+        null,
+        ingestionSpec,
+        Collections.emptyMap(),
+        indexingServiceClient
+    );
+  }
+
+  private static class TestFirehose implements FiniteFirehoseFactory<StringInputRowParser, Integer>
+  {
+    private final List<Integer> ids;
+
+    TestFirehose(List<Integer> ids)
+    {
+      this.ids = ids;
+    }
+
+    @Override
+    public Stream<InputSplit<Integer>> getSplits()
+    {
+      return ids.stream().map(InputSplit::new);
+    }
+
+    @Override
+    public int getNumSplits()
+    {
+      return ids.size();
+    }
+
+    @Override
+    public FiniteFirehoseFactory<StringInputRowParser, Integer> withSplit(InputSplit<Integer> split)
+    {
+      return new TestFirehose(Collections.singletonList(split.get()));
+    }
+  }
+
+  private class TestSupervisorTask extends TestParallelIndexSupervisorTask
+  {
+    TestSupervisorTask(
+        String id,
+        TaskResource taskResource,
+        ParallelIndexIngestionSpec ingestionSchema,
+        Map<String, Object> context,
+        IndexingServiceClient indexingServiceClient
+    )
+    {
+      super(
+          id,
+          taskResource,
+          ingestionSchema,
+          context,
+          indexingServiceClient
+      );
+    }
+
+    @Override
+    public TaskStatus run(TaskToolbox toolbox) throws Exception
+    {
+      setRunner(
+          new TestRunner(
+              toolbox,
+              this,
+              indexingServiceClient
+          )
+      );
+      return TaskStatus.fromCode(
+          getId(),
+          getRunner().run()
+      );
+    }
+  }
+
+  private class TestRunner extends TestParallelIndexTaskRunner
+  {
+    private final ParallelIndexSupervisorTask supervisorTask;
+
+    TestRunner(
+        TaskToolbox toolbox,
+        ParallelIndexSupervisorTask supervisorTask,
+        @Nullable IndexingServiceClient indexingServiceClient
+    )
+    {
+      super(
+          toolbox,
+          supervisorTask.getId(),
+          supervisorTask.getGroupId(),
+          supervisorTask.getIngestionSchema(),
+          supervisorTask.getContext(),
+          indexingServiceClient
+      );
+      this.supervisorTask = supervisorTask;
+    }
+
+    @Override
+    ParallelIndexSubTaskSpec newTaskSpec(InputSplit split)
+    {
+      final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema()
+          .getIOConfig()
+          .getFirehoseFactory();
+      final TestSubTaskSpec spec = new TestSubTaskSpec(
+          supervisorTask.getId() + "_" + getAndIncrementNextSpecId(),
+          supervisorTask.getGroupId(),
+          supervisorTask,
+          this,
+          new ParallelIndexIngestionSpec(
+              getIngestionSchema().getDataSchema(),
+              new ParallelIndexIOConfig(
+                  baseFirehoseFactory.withSplit(split),
+                  getIngestionSchema().getIOConfig().isAppendToExisting()
+              ),
+              getIngestionSchema().getTuningConfig()
+          ),
+          supervisorTask.getContext(),
+          split
+      );
+      subTaskSpecs.put(spec.getId(), spec);
+      return spec;
+    }
+  }
+
+  private class TestSubTaskSpec extends ParallelIndexSubTaskSpec
+  {
+    private final ParallelIndexSupervisorTask supervisorTask;
+
+    TestSubTaskSpec(
+        String id,
+        String groupId,
+        ParallelIndexSupervisorTask supervisorTask,
+        SinglePhaseParallelIndexTaskRunner runner,
+        ParallelIndexIngestionSpec ingestionSpec,
+        Map<String, Object> context,
+        InputSplit inputSplit
+    )
+    {
+      super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit);
+      this.supervisorTask = supervisorTask;
+    }
+
+    @Override
+    public ParallelIndexSubTask newSubTask(int numAttempts)
+    {
+      try {
+        // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id
+        Thread.sleep(10);
+      }
+      catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+      final TestSubTask subTask = new TestSubTask(
+          getGroupId(),
+          getSupervisorTaskId(),
+          numAttempts,
+          getIngestionSpec(),
+          getContext(),
+          new LocalParallelIndexTaskClientFactory(supervisorTask)
+      );
+      final TestFirehose firehose = (TestFirehose) getIngestionSpec().getIOConfig().getFirehoseFactory();
+      final InputSplit<Integer> split = firehose.getSplits().findFirst().orElse(null);
+      if (split == null) {
+        throw new ISE("Split is null");
+      }
+      runningTasks.add(subTask);
+      taskIdToSpec.put(subTask.getId(), this);
+      runningSpecs.put(
+          getId(),
+          new TaskStatusPlus(
+              subTask.getId(),
+              subTask.getType(),
+              DateTimes.EPOCH,
+              DateTimes.EPOCH,
+              TaskState.RUNNING,
+              RunnerTaskState.RUNNING,
+              -1L,
+              TaskLocation.unknown(),
+              null,
+              null
+          )
+      );
+      return subTask;
+    }
+  }
+
+  private class TestSubTask extends ParallelIndexSubTask
+  {
+    private volatile TaskState state = TaskState.RUNNING;
+
+    TestSubTask(
+        String groupId,
+        String supervisorTaskId,
+        int numAttempts,
+        ParallelIndexIngestionSpec ingestionSchema,
+        Map<String, Object> context,
+        IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory
+    )
+    {
+      super(
+          null,
+          groupId,
+          null,
+          supervisorTaskId,
+          numAttempts,
+          ingestionSchema,
+          context,
+          null,
+          taskClientFactory
+      );
+    }
+
+    @Override
+    public boolean isReady(TaskActionClient taskActionClient)
+    {
+      return true;
+    }
+
+    @Override
+    public TaskStatus run(final TaskToolbox toolbox) throws Exception
+    {
+      while (state == TaskState.RUNNING) {
+        Thread.sleep(100);
+      }
+
+      final TestFirehose firehose = (TestFirehose) getIngestionSchema().getIOConfig().getFirehoseFactory();
+
+      final List<TaskLock> locks = toolbox.getTaskActionClient()
+                                          .submit(new SurrogateAction<>(getSupervisorTaskId(), new LockListAction()));
+      Preconditions.checkState(locks.size() == 1, "There should be a single lock");
+
+      task.getRunner().collectReport(
+          new PushedSegmentsReport(
+              getId(),
+              Collections.singletonList(
+                  new DataSegment(
+                      getDataSource(),
+                      Intervals.of("2017/2018"),
+                      locks.get(0).getVersion(),
+                      null,
+                      null,
+                      null,
+                      new NumberedShardSpec(firehose.ids.get(0), NUM_SUB_TASKS),
+                      0,
+                      1L
+                  )
+              )
+          )
+      );
+      return TaskStatus.fromCode(getId(), state);
+    }
+
+    void setState(TaskState state)
+    {
+      Preconditions.checkArgument(
+          state == TaskState.SUCCESS || state == TaskState.FAILED,
+          "state[%s] should be SUCCESS of FAILED",
+          state
+      );
+      this.state = state;
+      final int taskIndex = IntStream.range(0, runningTasks.size())
+                                     .filter(i -> runningTasks.get(i).getId().equals(getId())).findAny()
+                                     .orElse(-1);
+      if (taskIndex == -1) {
+        throw new ISE("Can't find an index for task[%s]", getId());
+      }
+      runningTasks.remove(taskIndex);
+      final String specId = Preconditions.checkNotNull(taskIdToSpec.get(getId()), "spec for task[%s]", getId()).getId();
+      runningSpecs.remove(specId);
+      taskHistories.computeIfAbsent(specId, k -> new ArrayList<>()).add(
+          new TaskStatusPlus(
+              getId(),
+              getType(),
+              DateTimes.EPOCH,
+              DateTimes.EPOCH,
+              state,
+              RunnerTaskState.NONE,
+              -1L,
+              TaskLocation.unknown(),
+              null,
+              null
+          )
+      );
+    }
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java
new file mode 100644
index 0000000..9ace497
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.google.common.collect.Lists;
+import io.druid.client.indexing.NoopIndexingServiceClient;
+import io.druid.data.input.impl.CSVParseSpec;
+import io.druid.data.input.impl.DimensionsSpec;
+import io.druid.data.input.impl.ParseSpec;
+import io.druid.data.input.impl.StringInputRowParser;
+import io.druid.data.input.impl.TimestampSpec;
+import io.druid.indexing.common.TestUtils;
+import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
+import io.druid.indexing.common.task.Task;
+import io.druid.indexing.common.task.TaskResource;
+import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.granularity.Granularities;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.LongSumAggregatorFactory;
+import io.druid.segment.indexing.DataSchema;
+import io.druid.segment.indexing.granularity.UniformGranularitySpec;
+import io.druid.segment.realtime.firehose.LocalFirehoseFactory;
+import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
+import io.druid.server.security.AuthorizerMapper;
+import org.joda.time.Interval;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class ParallelIndexSupervisorTaskSerdeTest
+{
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec(
+      new TimestampSpec(
+          "ts",
+          "auto",
+          null
+      ),
+      new DimensionsSpec(
+          DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")),
+          Lists.newArrayList(),
+          Lists.newArrayList()
+      ),
+      null,
+      Arrays.asList("ts", "dim", "val"),
+      false,
+      0
+  );
+
+  private final TestUtils testUtils = new TestUtils();
+
+  @Test
+  public void testSerde() throws IOException
+  {
+    final ObjectMapper objectMapper = testUtils.getTestObjectMapper();
+    objectMapper.registerSubtypes(
+        new NamedType(LocalFirehoseFactory.class, "local")
+    );
+
+    final ParallelIndexSupervisorTask task = newTask(
+        objectMapper,
+        Intervals.of("2018/2019")
+    );
+    final String json = objectMapper.writeValueAsString(task);
+    Assert.assertEquals(task, objectMapper.readValue(json, Task.class));
+  }
+
+  private ParallelIndexSupervisorTask newTask(
+      ObjectMapper objectMapper,
+      Interval interval
+  )
+  {
+    // set up ingestion spec
+    final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
+        new DataSchema(
+            "dataSource",
+            objectMapper.convertValue(
+                new StringInputRowParser(
+                    DEFAULT_PARSE_SPEC,
+                    null
+                ),
+                Map.class
+            ),
+            new AggregatorFactory[]{
+                new LongSumAggregatorFactory("val", "val")
+            },
+            new UniformGranularitySpec(
+                Granularities.DAY,
+                Granularities.MINUTE,
+                interval == null ? null : Collections.singletonList(interval)
+            ),
+            null,
+            objectMapper
+        ),
+        new ParallelIndexIOConfig(
+            new LocalFirehoseFactory(new File("tmp"), "test_*", null),
+            false
+        ),
+        new ParallelIndexTuningConfig(
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            2,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null
+        )
+    );
+
+    // set up test tools
+    return new ParallelIndexSupervisorTask(
+        "taskId",
+        new TaskResource("group", 1),
+        ingestionSpec,
+        new HashMap<>(),
+        new NoopIndexingServiceClient(),
+        new NoopChatHandlerProvider(),
+        new AuthorizerMapper(Collections.emptyMap()),
+        new DropwizardRowIngestionMetersFactory()
+    );
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java
new file mode 100644
index 0000000..b20436a
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java
@@ -0,0 +1,370 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.data.input.FiniteFirehoseFactory;
+import io.druid.data.input.InputSplit;
+import io.druid.data.input.impl.StringInputRowParser;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.actions.TaskActionClient;
+import io.druid.indexing.common.task.TaskResource;
+import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.granularity.Granularities;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.LongSumAggregatorFactory;
+import io.druid.segment.indexing.DataSchema;
+import io.druid.segment.indexing.granularity.UniformGranularitySpec;
+import io.druid.segment.realtime.firehose.LocalFirehoseFactory;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.io.Writer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest
+{
+  private File inputDir;
+
+  @Before
+  public void setup() throws IOException
+  {
+    inputDir = temporaryFolder.newFolder("data");
+    // set up data
+    for (int i = 0; i < 5; i++) {
+      try (final Writer writer =
+               Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) {
+        writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 24 + i, i));
+        writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i));
+      }
+    }
+
+    for (int i = 0; i < 5; i++) {
+      try (final Writer writer =
+               Files.newBufferedWriter(new File(inputDir, "filtered_" + i).toPath(), StandardCharsets.UTF_8)) {
+        writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i));
+      }
+    }
+
+    indexingServiceClient = new LocalIndexingServiceClient();
+    localDeepStorage = temporaryFolder.newFolder("localStorage");
+  }
+
+  @After
+  public void teardown()
+  {
+    indexingServiceClient.shutdown();
+    temporaryFolder.delete();
+  }
+
+  @Test
+  public void testIsReady() throws Exception
+  {
+    final ParallelIndexSupervisorTask task = newTask(
+        Intervals.of("2017/2018"),
+        new ParallelIndexIOConfig(
+            new LocalFirehoseFactory(inputDir, "test_*", null),
+            false
+        )
+    );
+    actionClient = createActionClient(task);
+    toolbox = createTaskToolbox(task);
+
+    prepareTaskForLocking(task);
+    Assert.assertTrue(task.isReady(actionClient));
+
+    final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.createRunner(toolbox);
+    final Iterator<ParallelIndexSubTaskSpec> subTaskSpecIterator = runner.subTaskSpecIterator().iterator();
+
+    while (subTaskSpecIterator.hasNext()) {
+      final ParallelIndexSubTaskSpec spec = subTaskSpecIterator.next();
+      final ParallelIndexSubTask subTask = new ParallelIndexSubTask(
+          null,
+          spec.getGroupId(),
+          null,
+          spec.getSupervisorTaskId(),
+          0,
+          spec.getIngestionSpec(),
+          spec.getContext(),
+          indexingServiceClient,
+          null
+      );
+      final TaskActionClient subTaskActionClient = createActionClient(subTask);
+      prepareTaskForLocking(subTask);
+      Assert.assertTrue(subTask.isReady(subTaskActionClient));
+    }
+  }
+
+  @Test
+  public void testWithoutInterval() throws Exception
+  {
+    final ParallelIndexSupervisorTask task = newTask(
+        null,
+        new ParallelIndexIOConfig(
+            new LocalFirehoseFactory(inputDir, "test_*", null),
+            false
+        )
+    );
+    actionClient = createActionClient(task);
+    toolbox = createTaskToolbox(task);
+
+    prepareTaskForLocking(task);
+    Assert.assertTrue(task.isReady(actionClient));
+    Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode());
+  }
+
+  @Test()
+  public void testRunInParallel() throws Exception
+  {
+    final ParallelIndexSupervisorTask task = newTask(
+        Intervals.of("2017/2018"),
+        new ParallelIndexIOConfig(
+            new LocalFirehoseFactory(inputDir, "test_*", null),
+            false
+        )
+    );
+    actionClient = createActionClient(task);
+    toolbox = createTaskToolbox(task);
+
+    prepareTaskForLocking(task);
+    Assert.assertTrue(task.isReady(actionClient));
+    Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode());
+  }
+
+  @Test
+  public void testRunInSequential() throws Exception
+  {
+    final ParallelIndexSupervisorTask task = newTask(
+        Intervals.of("2017/2018"),
+        new ParallelIndexIOConfig(
+            new LocalFirehoseFactory(inputDir, "test_*", null)
+            {
+              @Override
+              public boolean isSplittable()
+              {
+                return false;
+              }
+            },
+            false
+        )
+    );
+    actionClient = createActionClient(task);
+    toolbox = createTaskToolbox(task);
+
+    prepareTaskForLocking(task);
+    Assert.assertTrue(task.isReady(actionClient));
+    Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode());
+  }
+
+  private ParallelIndexSupervisorTask newTask(
+      Interval interval,
+      ParallelIndexIOConfig ioConfig
+  )
+  {
+    // set up ingestion spec
+    final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
+        new DataSchema(
+            "dataSource",
+            getObjectMapper().convertValue(
+                new StringInputRowParser(
+                    DEFAULT_PARSE_SPEC,
+                    null
+                ),
+                Map.class
+            ),
+            new AggregatorFactory[]{
+                new LongSumAggregatorFactory("val", "val")
+            },
+            new UniformGranularitySpec(
+                Granularities.DAY,
+                Granularities.MINUTE,
+                interval == null ? null : Collections.singletonList(interval)
+            ),
+            null,
+            getObjectMapper()
+        ),
+        ioConfig,
+        new ParallelIndexTuningConfig(
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            2,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null
+        )
+    );
+
+    // set up test tools
+    return new TestSupervisorTask(
+        null,
+        null,
+        ingestionSpec,
+        new HashMap<>(),
+        indexingServiceClient
+    );
+  }
+
+  private static class TestSupervisorTask extends TestParallelIndexSupervisorTask
+  {
+    private final IndexingServiceClient indexingServiceClient;
+
+    TestSupervisorTask(
+        String id,
+        TaskResource taskResource,
+        ParallelIndexIngestionSpec ingestionSchema,
+        Map<String, Object> context,
+        IndexingServiceClient indexingServiceClient
+    )
+    {
+      super(
+          id,
+          taskResource,
+          ingestionSchema,
+          context,
+          indexingServiceClient
+      );
+      this.indexingServiceClient = indexingServiceClient;
+    }
+
+    @Override
+    public TaskStatus run(TaskToolbox toolbox) throws Exception
+    {
+      setToolbox(toolbox);
+      setRunner(
+          new TestRunner(
+              toolbox,
+              this,
+              indexingServiceClient
+          )
+      );
+      return TaskStatus.fromCode(
+          getId(),
+          getRunner().run()
+      );
+    }
+  }
+
+  private static class TestRunner extends TestParallelIndexTaskRunner
+  {
+    private final ParallelIndexSupervisorTask supervisorTask;
+
+    TestRunner(
+        TaskToolbox toolbox,
+        ParallelIndexSupervisorTask supervisorTask,
+        @Nullable IndexingServiceClient indexingServiceClient
+    )
+    {
+      super(
+          toolbox,
+          supervisorTask.getId(),
+          supervisorTask.getGroupId(),
+          supervisorTask.getIngestionSchema(),
+          supervisorTask.getContext(),
+          indexingServiceClient
+      );
+      this.supervisorTask = supervisorTask;
+    }
+
+    @Override
+    ParallelIndexSubTaskSpec newTaskSpec(InputSplit split)
+    {
+      final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema()
+          .getIOConfig()
+          .getFirehoseFactory();
+      return new TestParallelIndexSubTaskSpec(
+          supervisorTask.getId() + "_" + getAndIncrementNextSpecId(),
+          supervisorTask.getGroupId(),
+          supervisorTask,
+          new ParallelIndexIngestionSpec(
+              getIngestionSchema().getDataSchema(),
+              new ParallelIndexIOConfig(
+                  baseFirehoseFactory.withSplit(split),
+                  getIngestionSchema().getIOConfig().isAppendToExisting()
+              ),
+              getIngestionSchema().getTuningConfig()
+          ),
+          supervisorTask.getContext(),
+          split
+      );
+    }
+  }
+
+  private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSpec
+  {
+    private final ParallelIndexSupervisorTask supervisorTask;
+
+    TestParallelIndexSubTaskSpec(
+        String id,
+        String groupId,
+        ParallelIndexSupervisorTask supervisorTask,
+        ParallelIndexIngestionSpec ingestionSpec,
+        Map<String, Object> context,
+        InputSplit inputSplit
+    )
+    {
+      super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit);
+      this.supervisorTask = supervisorTask;
+    }
+
+    @Override
+    public ParallelIndexSubTask newSubTask(int numAttempts)
+    {
+      return new ParallelIndexSubTask(
+          null,
+          getGroupId(),
+          null,
+          getSupervisorTaskId(),
+          numAttempts,
+          getIngestionSpec(),
+          getContext(),
+          null,
+          new LocalParallelIndexTaskClientFactory(supervisorTask)
+      );
+    }
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java
new file mode 100644
index 0000000..33173b3
--- /dev/null
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.indexing.common.task.batch.parallel;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.client.indexing.NoopIndexingServiceClient;
+import io.druid.client.indexing.TaskStatusResponse;
+import io.druid.data.input.InputSplit;
+import io.druid.indexer.RunnerTaskState;
+import io.druid.indexer.TaskLocation;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatus;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.TaskToolbox;
+import io.druid.indexing.common.task.NoopTask;
+import io.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent;
+import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.concurrent.Execs;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class TaskMonitorTest
+{
+  private static final int SPLIT_NUM = 10;
+
+  private final ExecutorService taskRunner = Execs.multiThreaded(5, "task-monitor-test-%d");
+  private final ConcurrentMap<String, TaskState> tasks = new ConcurrentHashMap<>();
+  private final TaskMonitor<TestTask> monitor = new TaskMonitor<>(new TestIndexingServiceClient(), 3, SPLIT_NUM);
+
+  @Before
+  public void setup()
+  {
+    tasks.clear();
+    monitor.start(100);
+  }
+
+  @After
+  public void teardown()
+  {
+    monitor.stop();
+    taskRunner.shutdownNow();
+  }
+
+  @Test
+  public void testBasic() throws InterruptedException, ExecutionException, TimeoutException
+  {
+    final List<ListenableFuture<SubTaskCompleteEvent<TestTask>>> futures = IntStream
+        .range(0, 10)
+        .mapToObj(i -> monitor.submit(
+            new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, new IntegerInputSplit(i), 100L, 0)
+        ))
+        .collect(Collectors.toList());
+    for (int i = 0; i < futures.size(); i++) {
+      // # of threads of taskRunner is 5, so the expected max timeout is 2 sec. We additionally wait three more seconds
+      // here to make sure the test passes.
+      final SubTaskCompleteEvent<TestTask> result = futures.get(i).get(1, TimeUnit.SECONDS);
+      Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId());
+      Assert.assertEquals("specId" + i, result.getSpec().getId());
+      Assert.assertNotNull(result.getLastStatus());
+      Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState());
+      Assert.assertEquals(TaskState.SUCCESS, result.getLastState());
+    }
+  }
+
+  @Test
+  public void testRetry() throws InterruptedException, ExecutionException, TimeoutException
+  {
+    final List<TestTaskSpec> specs = IntStream
+        .range(0, 10)
+        .mapToObj(
+            i -> new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, new IntegerInputSplit(i), 100L, 2)
+        )
+        .collect(Collectors.toList());
+    final List<ListenableFuture<SubTaskCompleteEvent<TestTask>>> futures = specs
+        .stream()
+        .map(monitor::submit)
+        .collect(Collectors.toList());
+    for (int i = 0; i < futures.size(); i++) {
+      // # of threads of taskRunner is 5, and each task is expected to be run 3 times (with 2 retries), so the expected
+      // max timeout is 6 sec. We additionally wait 4 more seconds here to make sure the test passes.
+      final SubTaskCompleteEvent<TestTask> result = futures.get(i).get(2, TimeUnit.SECONDS);
+      Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId());
+      Assert.assertEquals("specId" + i, result.getSpec().getId());
+
+      Assert.assertNotNull(result.getLastStatus());
+      Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState());
+      Assert.assertEquals(TaskState.SUCCESS, result.getLastState());
+
+      final TaskHistory<TestTask> taskHistory = monitor.getCompleteSubTaskSpecHistory(specs.get(i).getId());
+      Assert.assertNotNull(taskHistory);
+
+      final List<TaskStatusPlus> attemptHistory = taskHistory.getAttemptHistory();
+      Assert.assertNotNull(attemptHistory);
+      Assert.assertEquals(3, attemptHistory.size());
+      Assert.assertEquals(TaskState.FAILED, attemptHistory.get(0).getState());
+      Assert.assertEquals(TaskState.FAILED, attemptHistory.get(1).getState());
+    }
+  }
+
+  private static class TestTaskSpec extends SubTaskSpec<TestTask>
+  {
+    private final long runTime;
+    private final int numMaxFails;
+
+    private int numFails;
+
+    public TestTaskSpec(
+        String id,
+        String groupId,
+        String supervisorTaskId,
+        Map<String, Object> context,
+        InputSplit inputSplit,
+        long runTime,
+        int numMaxFails
+    )
+    {
+      super(id, groupId, supervisorTaskId, context, inputSplit);
+      this.runTime = runTime;
+      this.numMaxFails = numMaxFails;
+    }
+
+    @Override
+    public TestTask newSubTask(int numAttempts)
+    {
+      return new TestTask(getId(), runTime, numFails++ < numMaxFails);
+    }
+  }
+
+  private static class TestTask extends NoopTask
+  {
+    private final boolean shouldFail;
+
+    TestTask(String id, long runTime, boolean shouldFail)
+    {
+      super(id, "testDataSource", runTime, 0, null, null, null);
+      this.shouldFail = shouldFail;
+    }
+
+    @Override
+    public TaskStatus run(TaskToolbox toolbox) throws Exception
+    {
+      if (shouldFail) {
+        Thread.sleep(getRunTime());
+        return TaskStatus.failure(getId());
+      } else {
+        return super.run(toolbox);
+      }
+    }
+  }
+
+  private class TestIndexingServiceClient extends NoopIndexingServiceClient
+  {
+    @Override
+    public String runTask(Object taskObject)
+    {
+      final TestTask task = (TestTask) taskObject;
+      tasks.put(task.getId(), TaskState.RUNNING);
+      taskRunner.submit(() -> tasks.put(task.getId(), task.run(null).getStatusCode()));
+      return task.getId();
+    }
+
+    @Override
+    public TaskStatusResponse getTaskStatus(String taskId)
+    {
+      return new TaskStatusResponse(
+          taskId,
+          new TaskStatusPlus(
+              taskId,
+              "testTask",
+              DateTimes.EPOCH,
+              DateTimes.EPOCH,
+              tasks.get(taskId),
+              RunnerTaskState.RUNNING,
+              -1L,
+              TaskLocation.unknown(),
+              "testDataSource",
+              null
+          )
+      );
+    }
+  }
+
+  private static class IntegerInputSplit extends InputSplit<Integer>
+  {
+    public IntegerInputSplit(int split)
+    {
+      super(split);
+    }
+  }
+}
diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
index 35a23eb..f2085ea 100644
--- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
@@ -41,6 +41,7 @@ import io.druid.data.input.impl.TimestampSpec;
 import io.druid.guice.GuiceAnnotationIntrospector;
 import io.druid.guice.GuiceInjectableValues;
 import io.druid.guice.GuiceInjectors;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.common.SegmentLoaderFactory;
 import io.druid.indexing.common.TaskToolboxFactory;
 import io.druid.indexing.common.TestUtils;
@@ -215,7 +216,14 @@ public class IngestSegmentFirehoseFactoryTest
     };
     final LocalTaskActionClientFactory tac = new LocalTaskActionClientFactory(
         TASK_STORAGE,
-        new TaskActionToolbox(TASK_LOCKBOX, mdc, newMockEmitter(), EasyMock.createMock(SupervisorManager.class))
+        new TaskActionToolbox(
+            TASK_LOCKBOX,
+            TASK_STORAGE,
+            mdc,
+            newMockEmitter(),
+            EasyMock.createMock(SupervisorManager.class),
+            new Counters()
+        )
     );
     SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
     EasyMock.replay(notifierFactory);
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
index 6319a86..615fe39 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
@@ -44,6 +44,7 @@ import io.druid.discovery.DataNodeService;
 import io.druid.discovery.DruidNodeAnnouncer;
 import io.druid.discovery.LookupNodeService;
 import io.druid.indexer.TaskState;
+import io.druid.indexing.common.Counters;
 import io.druid.indexing.common.SegmentLoaderFactory;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexer.TaskStatus;
@@ -527,8 +528,17 @@ public class TaskLifecycleTest
     Preconditions.checkNotNull(emitter);
 
     taskLockbox = new TaskLockbox(taskStorage);
-    tac = new LocalTaskActionClientFactory(taskStorage, new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock(
-        SupervisorManager.class)));
+    tac = new LocalTaskActionClientFactory(
+        taskStorage,
+        new TaskActionToolbox(
+            taskLockbox,
+            taskStorage,
+            mdc,
+            emitter,
+            EasyMock.createMock(SupervisorManager.class),
+            new Counters()
+        )
+    );
     File tmpDir = temporaryFolder.newFolder();
     taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null);
 
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java
index 3a02c8b..40588ce 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java
@@ -23,8 +23,10 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.indexer.RunnerTaskState;
 import io.druid.indexer.TaskInfo;
 import io.druid.indexer.TaskLocation;
+import io.druid.indexer.TaskState;
 import io.druid.indexer.TaskStatus;
 import io.druid.indexer.TaskStatusPlus;
 import io.druid.indexing.common.TaskToolbox;
@@ -860,11 +862,17 @@ public class OverlordResourceTest
   public void testGetTaskStatus() throws Exception
   {
     expectAuthorizationTokenCheck();
-    EasyMock.expect(taskStorageQueryAdapter.getStatus("mytask"))
-            .andReturn(Optional.of(TaskStatus.success("mytask")));
+    final Task task = NoopTask.create("mytask", 0);
+    final TaskStatus status = TaskStatus.running("mytask");
 
-    EasyMock.expect(taskStorageQueryAdapter.getStatus("othertask"))
-            .andReturn(Optional.absent());
+    EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("mytask"))
+            .andReturn(new TaskInfo<>(task.getId(), DateTimes.of("2018-01-01"), status, task.getDataSource(), task));
+
+    EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("othertask"))
+            .andReturn(null);
+
+    EasyMock.<Collection<? extends TaskRunnerWorkItem>>expect(taskRunner.getKnownTasks())
+        .andReturn(ImmutableList.of());
 
     EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req);
 
@@ -873,7 +881,24 @@ public class OverlordResourceTest
         TestHelper.makeJsonMapper().writeValueAsString(response1.getEntity()),
         TaskStatusResponse.class
     );
-    Assert.assertEquals(new TaskStatusResponse("mytask", TaskStatus.success("mytask")), taskStatusResponse1);
+    Assert.assertEquals(
+        new TaskStatusResponse(
+            "mytask",
+            new TaskStatusPlus(
+                "mytask",
+                "noop",
+                DateTimes.of("2018-01-01"),
+                DateTimes.EPOCH,
+                TaskState.RUNNING,
+                RunnerTaskState.RUNNING,
+                -1L,
+                TaskLocation.unknown(),
+                task.getDataSource(),
+                null
+            )
+        ),
+        taskStatusResponse1
+    );
 
     final Response response2 = overlordResource.getTaskStatus("othertask");
     final TaskStatusResponse taskStatusResponse2 = TestHelper.makeJsonMapper().readValue(
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java
index 4c08cce..3b4d6a7 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java
@@ -248,7 +248,7 @@ public class OverlordTest
     Assert.assertEquals(taskId_0, ((TaskStatusResponse) response.getEntity()).getTask());
     Assert.assertEquals(
         TaskStatus.running(taskId_0).getStatusCode(),
-        ((TaskStatusResponse) response.getEntity()).getStatus().getStatusCode()
+        ((TaskStatusResponse) response.getEntity()).getStatus().getState()
     );
 
     // Simulate completion of task_0
@@ -296,7 +296,7 @@ public class OverlordTest
   {
     while (true) {
       Response response = overlordResource.getTaskStatus(taskId);
-      if (status.equals(((TaskStatusResponse) response.getEntity()).getStatus().getStatusCode())) {
+      if (status.equals(((TaskStatusResponse) response.getEntity()).getStatus().getState())) {
         break;
       }
       Thread.sleep(10);
diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java
index d0e9b4e..c3df10e 100644
--- a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java
+++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Predicates;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
+import io.druid.client.indexing.TaskStatusResponse;
 import io.druid.indexer.TaskState;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.RetryUtils;
@@ -125,12 +126,13 @@ public class OverlordResourceTestClient
       );
 
       LOG.info("Index status response" + response.getContent());
-      Map<String, Object> responseData = jsonMapper.readValue(
-          response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
+      TaskStatusResponse taskStatusResponse = jsonMapper.readValue(
+          response.getContent(),
+          new TypeReference<TaskStatusResponse>()
+          {
+          }
       );
-      //TODO: figure out a better way to parse the response...
-      String status = (String) ((Map) responseData.get("status")).get("status");
-      return TaskState.valueOf(status);
+      return taskStatusResponse.getStatus().getState();
     }
     catch (Exception e) {
       throw Throwables.propagate(e);
diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java
new file mode 100644
index 0000000..74cfb7e
--- /dev/null
+++ b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.tests.indexer;
+
+import com.google.inject.Inject;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.testing.IntegrationTestingConfig;
+import io.druid.testing.clients.ClientInfoResourceTestClient;
+import io.druid.testing.utils.RetryUtil;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.util.List;
+
+public class AbstractITBatchIndexTest extends AbstractIndexerTest
+{
+  private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class);
+
+  @Inject
+  IntegrationTestingConfig config;
+
+  @Inject
+  ClientInfoResourceTestClient clientInfoResourceTestClient;
+
+  void doIndexTestTest(
+      String dataSource,
+      String indexTaskFilePath,
+      String queryFilePath
+  ) throws IOException
+  {
+    submitTaskAndWait(indexTaskFilePath, dataSource);
+    try {
+      queryHelper.testQueriesFromFile(queryFilePath, 2);
+
+    }
+    catch (Exception e) {
+      LOG.error(e, "Error while testing");
+      throw new RuntimeException(e);
+    }
+  }
+
+  void doReindexTest(
+      String reindexDataSource,
+      String reindexTaskFilePath,
+      String queryFilePath
+  ) throws IOException
+  {
+    submitTaskAndWait(reindexTaskFilePath, reindexDataSource);
+    try {
+      queryHelper.testQueriesFromFile(queryFilePath, 2);
+      // verify excluded dimension is not reIndexed
+      final List<String> dimensions = clientInfoResourceTestClient.getDimensions(
+          reindexDataSource,
+          "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z"
+      );
+      Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot"));
+    }
+    catch (Exception e) {
+      LOG.error(e, "Error while testing");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private void submitTaskAndWait(String indexTaskFilePath, String dataSourceName) throws IOException
+  {
+    final String taskID = indexer.submitTask(getTaskAsString(indexTaskFilePath));
+    LOG.info("TaskID for loading index task %s", taskID);
+    indexer.waitUntilTaskCompletes(taskID);
+
+    RetryUtil.retryUntilTrue(
+        () -> coordinator.areSegmentsLoaded(dataSourceName), "Segment Load"
+    );
+  }
+}
diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java
index d4fd5be..5195a6c 100644
--- a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java
+++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java
@@ -19,97 +19,37 @@
 
 package io.druid.tests.indexer;
 
-import com.google.common.base.Throwables;
-import com.google.inject.Inject;
-import io.druid.java.util.common.logger.Logger;
-import io.druid.testing.IntegrationTestingConfig;
-import io.druid.testing.clients.ClientInfoResourceTestClient;
 import io.druid.testing.guice.DruidTestModuleFactory;
-import io.druid.testing.utils.RetryUtil;
-import org.junit.Assert;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import java.util.List;
-import java.util.concurrent.Callable;
-
 @Guice(moduleFactory = DruidTestModuleFactory.class)
-public class ITIndexerTest extends AbstractIndexerTest
+public class ITIndexerTest extends AbstractITBatchIndexTest
 {
-  private static final Logger LOG = new Logger(ITIndexerTest.class);
   private static String INDEX_TASK = "/indexer/wikipedia_index_task.json";
   private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
   private static String INDEX_DATASOURCE = "wikipedia_index_test";
   private static String REINDEX_TASK = "/indexer/wikipedia_reindex_task.json";
   private static String REINDEX_DATASOURCE = "wikipedia_reindex_test";
 
-  @Inject
-  private IntegrationTestingConfig config;
-
-  @Inject
-  ClientInfoResourceTestClient clientInfoResourceTestClient;
-
   @Test
   public void testIndexData() throws Exception
   {
-    loadData();
     try {
-      queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2);
-      reIndexData();
-      queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2);
-      // verify excluded dimension is not reIndexed
-      final List<String> dimensions = clientInfoResourceTestClient.getDimensions(
+      doIndexTestTest(
+          INDEX_DATASOURCE,
+          INDEX_TASK,
+          INDEX_QUERIES_RESOURCE
+      );
+      doReindexTest(
           REINDEX_DATASOURCE,
-          "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z"
+          REINDEX_TASK,
+          INDEX_QUERIES_RESOURCE
       );
-      Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot"));
-    }
-    catch (Exception e) {
-      LOG.error(e, "Error while testing");
-      throw Throwables.propagate(e);
     }
     finally {
       unloadAndKillData(INDEX_DATASOURCE);
       unloadAndKillData(REINDEX_DATASOURCE);
-
     }
-
-  }
-
-  private void loadData() throws Exception
-  {
-    final String taskID = indexer.submitTask(getTaskAsString(INDEX_TASK));
-    LOG.info("TaskID for loading index task %s", taskID);
-    indexer.waitUntilTaskCompletes(taskID);
-
-    RetryUtil.retryUntilTrue(
-        new Callable<Boolean>()
-        {
-          @Override
-          public Boolean call()
-          {
-            return coordinator.areSegmentsLoaded(INDEX_DATASOURCE);
-          }
-        }, "Segment Load"
-    );
-  }
-
-  private void reIndexData() throws Exception
-  {
-    final String taskID = indexer.submitTask(getTaskAsString(REINDEX_TASK));
-    LOG.info("TaskID for loading index task %s", taskID);
-    indexer.waitUntilTaskCompletes(taskID);
-
-    RetryUtil.retryUntilTrue(
-        new Callable<Boolean>()
-        {
-          @Override
-          public Boolean call()
-          {
-            return coordinator.areSegmentsLoaded(REINDEX_DATASOURCE);
-          }
-        }, "Segment Load"
-    );
   }
-
 }
diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java
similarity index 50%
copy from server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java
copy to integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java
index 0324ce1..0dff81e 100644
--- a/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java
+++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java
@@ -17,22 +17,31 @@
  * under the License.
  */
 
-package io.druid.indexing.overlord.supervisor;
+package io.druid.tests.indexer;
 
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.druid.testing.guice.DruidTestModuleFactory;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
 
-import java.util.List;
-
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
-@JsonSubTypes(value = {
-    @JsonSubTypes.Type(name = "NoopSupervisorSpec", value = NoopSupervisorSpec.class)
-})
-public interface SupervisorSpec
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITParallelIndexTest extends AbstractITBatchIndexTest
 {
-  String getId();
-
-  Supervisor createSupervisor();
+  private static String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json";
+  private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json";
+  private static String INDEX_DATASOURCE = "wikipedia_parallel_index_test";
 
-  List<String> getDataSources();
+  @Test
+  public void testIndexData() throws Exception
+  {
+    try {
+      doIndexTestTest(
+          INDEX_DATASOURCE,
+          INDEX_TASK,
+          INDEX_QUERIES_RESOURCE
+      );
+    }
+    finally {
+      unloadAndKillData(INDEX_DATASOURCE);
+    }
+  }
 }
diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java
index 4397c47..11fe9ca 100644
--- a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java
+++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java
@@ -47,7 +47,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest
   private static final Logger LOG = new Logger(ITUnionQueryTest.class);
   private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json";
   private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName";
-  private static final String UNION_DATA_FILE = "/indexer/wikipedia_index_data.json";
+  private static final String UNION_DATA_FILE = "/data/union_query/wikipedia_index_data.json";
   private static final String UNION_QUERIES_RESOURCE = "/indexer/union_queries.json";
   private static final String UNION_DATASOURCE = "wikipedia_index_test";
 
@@ -91,7 +91,9 @@ public class ITUnionQueryTest extends AbstractIndexerTest
       RetryUtil.retryUntil(
           () -> {
             for (int i = 0; i < numTasks; i++) {
-              if (queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01") < 5) {
+              final int countRows = queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01");
+              if (countRows < 5) {
+                LOG.warn("%d events have been ingested to %s so far", countRows, UNION_DATASOURCE + i);
                 return false;
               }
             }
diff --git a/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json
new file mode 100644
index 0000000..169796c
--- /dev/null
+++ b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json
@@ -0,0 +1,3 @@
+{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
+{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
+{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
\ No newline at end of file
diff --git a/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json
new file mode 100644
index 0000000..62e2701
--- /dev/null
+++ b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json
@@ -0,0 +1,3 @@
+{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
+{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
+{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
\ No newline at end of file
diff --git a/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json
new file mode 100644
index 0000000..28e0762
--- /dev/null
+++ b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json
@@ -0,0 +1,4 @@
+{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
+{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
+{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
+{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
\ No newline at end of file
diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_data.json b/integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json
similarity index 100%
rename from integration-tests/src/test/resources/indexer/wikipedia_index_data.json
rename to integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json
diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json
index 15e0de8..af7b98a 100644
--- a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json
+++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json
@@ -58,8 +58,8 @@
             "type": "index",
             "firehose": {
                 "type": "local",
-                "baseDir": "/resources/indexer",
-                "filter": "wikipedia_index_data.json"
+                "baseDir": "/resources/data/batch_index",
+                "filter": "wikipedia_index_data*"
             }
         },
         "tuningConfig": {
diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_queries.json
new file mode 100644
index 0000000..76ecb5c
--- /dev/null
+++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_queries.json
@@ -0,0 +1,71 @@
+[
+    {
+        "description": "timeseries, 1 agg, all",
+        "query":{
+            "queryType" : "timeBoundary",
+            "dataSource": "wikipedia_parallel_index_test"
+        },
+        "expectedResults":[
+            {
+                "timestamp" : "2013-08-31T01:02:33.000Z",
+                "result" : {
+                    "minTime" : "2013-08-31T01:02:33.000Z",
+                    "maxTime" : "2013-09-01T12:41:27.000Z"
+                }
+            }
+        ]
+    },
+
+    {
+        "description":"having spec on post aggregation",
+        "query":{
+            "queryType":"groupBy",
+            "dataSource":"wikipedia_parallel_index_test",
+            "granularity":"day",
+            "dimensions":[
+                "page"
+            ],
+            "filter":{
+                "type":"selector",
+                "dimension":"language",
+                "value":"zh"
+            },
+            "aggregations":[
+                {
+                    "type":"count",
+                    "name":"rows"
+                },
+                {
+                    "type":"longSum",
+                    "fieldName":"added",
+                    "name":"added_count"
+                }
+            ],
+            "postAggregations": [
+                {
+                    "type":"arithmetic",
+                    "name":"added_count_times_ten",
+                    "fn":"*",
+                    "fields":[
+                        {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+                        {"type":"constant", "name":"const", "value":10}
+                    ]
+                }
+            ],
+            "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+            "intervals":[
+                "2013-08-31T00:00/2013-09-01T00:00"
+            ]
+        },
+        "expectedResults":[ {
+            "version" : "v1",
+            "timestamp" : "2013-08-31T00:00:00.000Z",
+            "event" : {
+                "added_count_times_ten" : 9050.0,
+                "page" : "Crimson Typhoon",
+                "added_count" : 905,
+                "rows" : 1
+            }
+        } ]
+    }
+]
\ No newline at end of file
diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json
similarity index 87%
copy from integration-tests/src/test/resources/indexer/wikipedia_index_task.json
copy to integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json
index 15e0de8..911adbd 100644
--- a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json
+++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json
@@ -1,8 +1,8 @@
 {
-    "type": "index",
+    "type": "index_parallel",
     "spec": {
         "dataSchema": {
-            "dataSource": "wikipedia_index_test",
+            "dataSource": "wikipedia_parallel_index_test",
             "metricsSpec": [
                 {
                     "type": "count",
@@ -55,15 +55,12 @@
             }
         },
         "ioConfig": {
-            "type": "index",
+            "type": "index_parallel",
             "firehose": {
                 "type": "local",
-                "baseDir": "/resources/indexer",
-                "filter": "wikipedia_index_data.json"
+                "baseDir": "/resources/data/batch_index",
+                "filter": "wikipedia_index_data*"
             }
-        },
-        "tuningConfig": {
-            "type": "index"
         }
     }
 }
\ No newline at end of file
diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java
similarity index 76%
copy from server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java
copy to server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java
index 4aaf523..b012db6 100644
--- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java
+++ b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java
@@ -47,13 +47,13 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-public class IndexingServiceClient
+public class HttpIndexingServiceClient implements IndexingServiceClient
 {
   private final DruidLeaderClient druidLeaderClient;
   private final ObjectMapper jsonMapper;
... 1330 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org