@Test
  public void testIndexTask() throws Exception {
    final Task indexTask =
        new IndexTask(
            null,
            null,
            new IndexTask.IndexIngestionSpec(
                new DataSchema(
                    "foo",
                    null,
                    new AggregatorFactory[] {new DoubleSumAggregatorFactory("met", "met")},
                    new UniformGranularitySpec(
                        Granularity.DAY, null, ImmutableList.of(new Interval("2010-01-01/P2D"))),
                    mapper),
                new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)),
                new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)),
            mapper,
            null);

    final Optional<TaskStatus> preRunTaskStatus = tsqa.getStatus(indexTask.getId());
    Assert.assertTrue("pre run task status not present", !preRunTaskStatus.isPresent());

    final TaskStatus mergedStatus = runTask(indexTask);
    final TaskStatus status = ts.getStatus(indexTask.getId()).get();
    final List<DataSegment> publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished());
    final List<DataSegment> loggedSegments =
        byIntervalOrdering.sortedCopy(tsqa.getInsertedSegments(indexTask.getId()));

    Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
    Assert.assertEquals(
        "merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode());
    Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments);
    Assert.assertEquals("num segments published", 2, mdc.getPublished().size());
    Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());

    Assert.assertEquals("segment1 datasource", "foo", publishedSegments.get(0).getDataSource());
    Assert.assertEquals(
        "segment1 interval",
        new Interval("2010-01-01/P1D"),
        publishedSegments.get(0).getInterval());
    Assert.assertEquals(
        "segment1 dimensions",
        ImmutableList.of("dim1", "dim2"),
        publishedSegments.get(0).getDimensions());
    Assert.assertEquals(
        "segment1 metrics", ImmutableList.of("met"), publishedSegments.get(0).getMetrics());

    Assert.assertEquals("segment2 datasource", "foo", publishedSegments.get(1).getDataSource());
    Assert.assertEquals(
        "segment2 interval",
        new Interval("2010-01-02/P1D"),
        publishedSegments.get(1).getInterval());
    Assert.assertEquals(
        "segment2 dimensions",
        ImmutableList.of("dim1", "dim2"),
        publishedSegments.get(1).getDimensions());
    Assert.assertEquals(
        "segment2 metrics", ImmutableList.of("met"), publishedSegments.get(1).getMetrics());
  }
  private TaskStatus runTask(final Task task) throws Exception {
    final Task dummyTask =
        new DefaultObjectMapper()
            .readValue("{\"type\":\"noop\", \"isReadyResult\":\"exception\"}\"", Task.class);
    final long startTime = System.currentTimeMillis();

    Preconditions.checkArgument(!task.getId().equals(dummyTask.getId()));

    tq.add(dummyTask);
    tq.add(task);

    TaskStatus retVal = null;

    for (final String taskId : ImmutableList.of(dummyTask.getId(), task.getId())) {
      try {
        TaskStatus status;
        while ((status = tsqa.getStatus(taskId).get()).isRunnable()) {
          if (System.currentTimeMillis() > startTime + 10 * 1000) {
            throw new ISE("Where did the task go?!: %s", task.getId());
          }

          Thread.sleep(100);
        }
        if (taskId.equals(task.getId())) {
          retVal = status;
        }
      } catch (Exception e) {
        throw Throwables.propagate(e);
      }
    }

    return retVal;
  }
  @Override
  public Set<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException {
    toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true);

    final Set<DataSegment> retVal =
        toolbox.getIndexerDBCoordinator().announceHistoricalSegments(segments);

    // Emit metrics
    final ServiceMetricEvent.Builder metricBuilder =
        new ServiceMetricEvent.Builder().setUser2(task.getDataSource()).setUser4(task.getType());

    for (DataSegment segment : segments) {
      metricBuilder.setUser5(segment.getInterval().toString());
      toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize()));
    }

    return retVal;
  }
    @Override
    public TaskStatus call() {
      final long startTime = System.currentTimeMillis();
      final File taskDir = toolbox.getTaskWorkDir();

      TaskStatus status;

      try {
        log.info("Running task: %s", task.getId());
        status = task.run(toolbox);
      } catch (InterruptedException e) {
        log.error(e, "Interrupted while running task[%s]", task);
        throw Throwables.propagate(e);
      } catch (Exception e) {
        log.error(e, "Exception while running task[%s]", task);
        status = TaskStatus.failure(task.getId());
      } catch (Throwable t) {
        log.error(t, "Uncaught Throwable while running task[%s]", task);
        throw Throwables.propagate(t);
      }

      try {
        if (taskDir.exists()) {
          log.info("Removing task directory: %s", taskDir);
          FileUtils.deleteDirectory(taskDir);
        }
      } catch (Exception e) {
        log.makeAlert(e, "Failed to delete task directory")
            .addData("taskDir", taskDir.toString())
            .addData("task", task.getId())
            .emit();
      }

      try {
        return status.withDuration(System.currentTimeMillis() - startTime);
      } catch (Exception e) {
        log.error(e, "Uncaught Exception during callback for task[%s]", task);
        throw Throwables.propagate(e);
      }
    }
  private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query) {
    QueryRunner<T> queryRunner = null;
    final String queryDataSource = Iterables.getOnlyElement(query.getDataSource().getNames());

    for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem :
        ImmutableList.copyOf(runningItems)) {
      final Task task = taskRunnerWorkItem.getTask();
      if (task.getDataSource().equals(queryDataSource)) {
        final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);

        if (taskQueryRunner != null) {
          if (queryRunner == null) {
            queryRunner = taskQueryRunner;
          } else {
            log.makeAlert("Found too many query runners for datasource")
                .addData("dataSource", queryDataSource)
                .emit();
          }
        }
      }
    }

    return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner;
  }
  @Override
  public Optional<ImmutableWorkerInfo> findWorkerForTask(
      final WorkerTaskRunnerConfig config,
      final ImmutableMap<String, ImmutableWorkerInfo> zkWorkers,
      final Task task) {
    // don't run other datasources on affinity workers; we only want our configured datasources to
    // run on them
    ImmutableMap.Builder<String, ImmutableWorkerInfo> builder = new ImmutableMap.Builder<>();
    for (String workerHost : zkWorkers.keySet()) {
      if (!affinityWorkerHosts.contains(workerHost)) {
        builder.put(workerHost, zkWorkers.get(workerHost));
      }
    }
    ImmutableMap<String, ImmutableWorkerInfo> eligibleWorkers = builder.build();

    List<String> workerHosts = affinityConfig.getAffinity().get(task.getDataSource());
    if (workerHosts == null) {
      return super.findWorkerForTask(config, eligibleWorkers, task);
    }

    ImmutableMap.Builder<String, ImmutableWorkerInfo> affinityBuilder =
        new ImmutableMap.Builder<>();
    for (String workerHost : workerHosts) {
      ImmutableWorkerInfo zkWorker = zkWorkers.get(workerHost);
      if (zkWorker != null) {
        affinityBuilder.put(workerHost, zkWorker);
      }
    }
    ImmutableMap<String, ImmutableWorkerInfo> affinityWorkers = affinityBuilder.build();

    if (!affinityWorkers.isEmpty()) {
      Optional<ImmutableWorkerInfo> retVal = super.findWorkerForTask(config, affinityWorkers, task);
      if (retVal.isPresent()) {
        return retVal;
      }
    }

    return super.findWorkerForTask(config, eligibleWorkers, task);
  }
 @Override
 public synchronized ListenableFuture<TaskStatus> run(final Task task) {
   final String taskId = task.getId();
   ListenableFuture<TaskStatus> future =
       MoreExecutors.listeningDecorator(Execs.singleThreaded("noop_test_task_exec_%s"))
           .submit(
               new Callable<TaskStatus>() {
                 @Override
                 public TaskStatus call() throws Exception {
                   // adding of task to list of runningTasks should be done before count down as
                   // getRunningTasks may not include the task for which latch has been counted
                   // down
                   // Count down to let know that task is actually running
                   // this is equivalent of getting process holder to run task in
                   // ForkingTaskRunner
                   runningTasks.add(taskId);
                   if (runLatches != null) {
                     runLatches[Integer.parseInt(taskId)].countDown();
                   }
                   // Wait for completion count down
                   if (completionLatches != null) {
                     completionLatches[Integer.parseInt(taskId)].await();
                   }
                   taskRunnerWorkItems.remove(taskId);
                   runningTasks.remove(taskId);
                   return TaskStatus.success(taskId);
                 }
               });
   TaskRunnerWorkItem taskRunnerWorkItem =
       new TaskRunnerWorkItem(taskId, future) {
         @Override
         public TaskLocation getLocation() {
           return TASK_LOCATION;
         }
       };
   taskRunnerWorkItems.put(taskId, taskRunnerWorkItem);
   return future;
 }
 private Task createMockTask(String type) {
   Task mock = EasyMock.createMock(Task.class);
   EasyMock.expect(mock.getType()).andReturn(type).anyTimes();
   EasyMock.replay(mock);
   return mock;
 }
 private ThreadPoolTaskRunnerWorkItem(Task task, ListenableFuture<TaskStatus> result) {
   super(task.getId(), result);
   this.task = task;
 }
  @Test
  public void testResumeTasks() throws Exception {
    final Task indexTask =
        new IndexTask(
            null,
            null,
            new IndexTask.IndexIngestionSpec(
                new DataSchema(
                    "foo",
                    null,
                    new AggregatorFactory[] {new DoubleSumAggregatorFactory("met", "met")},
                    new UniformGranularitySpec(
                        Granularity.DAY, null, ImmutableList.of(new Interval("2010-01-01/P2D"))),
                    mapper),
                new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)),
                new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)),
            mapper,
            null);

    final long startTime = System.currentTimeMillis();

    // manually insert the task into TaskStorage, waiting for TaskQueue to sync from storage
    ts.insert(indexTask, TaskStatus.running(indexTask.getId()));

    while (tsqa.getStatus(indexTask.getId()).get().isRunnable()) {
      if (System.currentTimeMillis() > startTime + 10 * 1000) {
        throw new ISE("Where did the task go?!: %s", indexTask.getId());
      }

      Thread.sleep(100);
    }

    final TaskStatus status = ts.getStatus(indexTask.getId()).get();
    final List<DataSegment> publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished());
    final List<DataSegment> loggedSegments =
        byIntervalOrdering.sortedCopy(tsqa.getInsertedSegments(indexTask.getId()));

    Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
    Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments);
    Assert.assertEquals("num segments published", 2, mdc.getPublished().size());
    Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());

    Assert.assertEquals("segment1 datasource", "foo", publishedSegments.get(0).getDataSource());
    Assert.assertEquals(
        "segment1 interval",
        new Interval("2010-01-01/P1D"),
        publishedSegments.get(0).getInterval());
    Assert.assertEquals(
        "segment1 dimensions",
        ImmutableList.of("dim1", "dim2"),
        publishedSegments.get(0).getDimensions());
    Assert.assertEquals(
        "segment1 metrics", ImmutableList.of("met"), publishedSegments.get(0).getMetrics());

    Assert.assertEquals("segment2 datasource", "foo", publishedSegments.get(1).getDataSource());
    Assert.assertEquals(
        "segment2 interval",
        new Interval("2010-01-02/P1D"),
        publishedSegments.get(1).getInterval());
    Assert.assertEquals(
        "segment2 dimensions",
        ImmutableList.of("dim1", "dim2"),
        publishedSegments.get(1).getDimensions());
    Assert.assertEquals(
        "segment2 metrics", ImmutableList.of("met"), publishedSegments.get(1).getMetrics());
  }