public static String printLogical(List<PlanFragment> fragments) {
    Map<PlanFragmentId, PlanFragment> fragmentsById =
        Maps.uniqueIndex(
            fragments,
            new Function<PlanFragment, PlanFragmentId>() {
              @Override
              public PlanFragmentId apply(PlanFragment input) {
                return input.getId();
              }
            });
    PlanNodeIdGenerator idGenerator = new PlanNodeIdGenerator();

    StringBuilder output = new StringBuilder();
    output.append("digraph logical_plan {\n");

    for (PlanFragment fragment : fragments) {
      printFragmentNodes(output, fragment, idGenerator);
    }

    for (PlanFragment fragment : fragments) {
      fragment.getRoot().accept(new EdgePrinter(output, fragmentsById, idGenerator), null);
    }

    output.append("}\n");

    return output.toString();
  }
    @Override
    public Void visitExchange(ExchangeNode node, Void context) {
      for (PlanFragmentId planFragmentId : node.getSourceFragmentIds()) {
        PlanFragment target = fragmentsById.get(planFragmentId);
        printEdge(node, target.getRoot());
      }

      return null;
    }
 private synchronized List<TaskSource> getSources() {
   return Stream.concat(
           Stream.of(planFragment.getPartitionedSourceNode()),
           planFragment.getRemoteSourceNodes().stream())
       .filter(Objects::nonNull)
       .map(PlanNode::getId)
       .map(this::getSource)
       .filter(Objects::nonNull)
       .collect(toImmutableList());
 }
  private static void printFragmentNodes(
      StringBuilder output, PlanFragment fragment, PlanNodeIdGenerator idGenerator) {
    String clusterId = "cluster_" + fragment.getId();
    output.append("subgraph ").append(clusterId).append(" {").append('\n');

    output.append(format("label = \"%s\"", fragment.getDistribution())).append('\n');

    PlanNode plan = fragment.getRoot();
    plan.accept(new NodePrinter(output, idGenerator), null);

    output.append("}").append('\n');
  }
  private static void printSubPlan(
      SubPlan plan,
      Map<PlanFragmentId, PlanFragment> fragmentsById,
      PlanNodeIdGenerator idGenerator,
      StringBuilder output) {
    PlanFragment fragment = plan.getFragment();
    printFragmentNodes(output, fragment, idGenerator);
    fragment.getRoot().accept(new EdgePrinter(output, fragmentsById, idGenerator), null);

    for (SubPlan child : plan.getChildren()) {
      printSubPlan(child, fragmentsById, idGenerator, output);
    }
  }
  @Override
  public synchronized void addSplits(PlanNodeId sourceId, Iterable<Split> splits) {
    try (SetThreadName ignored = new SetThreadName("HttpRemoteTask-%s", taskId)) {
      requireNonNull(sourceId, "sourceId is null");
      requireNonNull(splits, "splits is null");
      checkState(
          !noMoreSplits.contains(sourceId), "noMoreSplits has already been set for %s", sourceId);

      // only add pending split if not done
      if (!getTaskInfo().getState().isDone()) {
        int added = 0;
        for (Split split : splits) {
          if (pendingSplits.put(
              sourceId, new ScheduledSplit(nextSplitId.getAndIncrement(), split))) {
            added++;
          }
        }
        if (sourceId.equals(planFragment.getPartitionedSource())) {
          pendingSourceSplitCount += added;
          fireSplitCountChanged(added);
        }
        needsUpdate.set(true);
      }

      scheduleUpdate();
    }
  }
  public static String printDistributed(SubPlan plan) {
    List<PlanFragment> fragments = plan.getAllFragments();
    Map<PlanFragmentId, PlanFragment> fragmentsById =
        Maps.uniqueIndex(fragments, PlanFragment.idGetter());
    PlanNodeIdGenerator idGenerator = new PlanNodeIdGenerator();

    StringBuilder output = new StringBuilder();
    output.append("digraph distributed_plan {\n");

    printSubPlan(plan, fragmentsById, idGenerator, output);

    output.append("}\n");

    return output.toString();
  }
  private SqlTaskExecution(
      Session session,
      TaskId taskId,
      URI location,
      PlanFragment fragment,
      OutputBuffers outputBuffers,
      LocalExecutionPlanner planner,
      DataSize maxBufferSize,
      TaskExecutor taskExecutor,
      DataSize maxTaskMemoryUsage,
      DataSize operatorPreAllocatedMemory,
      QueryMonitor queryMonitor,
      Executor notificationExecutor,
      boolean cpuTimerEnabled) {
    try (SetThreadName setThreadName = new SetThreadName("Task-%s", taskId)) {
      this.taskId = checkNotNull(taskId, "taskId is null");
      this.location = checkNotNull(location, "location is null");
      this.taskExecutor = checkNotNull(taskExecutor, "driverExecutor is null");
      this.notificationExecutor =
          checkNotNull(notificationExecutor, "notificationExecutor is null");

      this.taskStateMachine = new TaskStateMachine(taskId, notificationExecutor);
      taskStateMachine.addStateChangeListener(
          new StateChangeListener<TaskState>() {
            @Override
            public void stateChanged(TaskState taskState) {
              if (taskState.isDone()) {
                SqlTaskExecution.this.taskExecutor.removeTask(taskHandle);
                // make sure buffers are cleaned up
                if (taskState != TaskState.FAILED) {
                  // don't close buffers for a failed query
                  // closed buffers signal to upstream tasks that everything finished cleanly
                  sharedBuffer.destroy();
                }
              }
            }
          });

      this.taskContext =
          new TaskContext(
              taskStateMachine,
              notificationExecutor,
              session,
              checkNotNull(maxTaskMemoryUsage, "maxTaskMemoryUsage is null"),
              checkNotNull(operatorPreAllocatedMemory, "operatorPreAllocatedMemory is null"),
              cpuTimerEnabled);

      this.sharedBuffer =
          new SharedBuffer(
              taskId,
              notificationExecutor,
              checkNotNull(maxBufferSize, "maxBufferSize is null"),
              outputBuffers);
      sharedBuffer.addStateChangeListener(
          new StateChangeListener<QueueState>() {
            @Override
            public void stateChanged(QueueState taskState) {
              if (taskState == QueueState.FINISHED) {
                checkTaskCompletion();
              }
            }
          });

      this.queryMonitor = checkNotNull(queryMonitor, "queryMonitor is null");

      taskHandle = taskExecutor.addTask(taskId);

      LocalExecutionPlan localExecutionPlan =
          planner.plan(
              session,
              fragment.getRoot(),
              fragment.getSymbols(),
              new TaskOutputFactory(sharedBuffer));
      List<DriverFactory> driverFactories = localExecutionPlan.getDriverFactories();

      // index driver factories
      DriverSplitRunnerFactory partitionedDriverFactory = null;
      ImmutableList.Builder<DriverSplitRunnerFactory> unpartitionedDriverFactories =
          ImmutableList.builder();
      for (DriverFactory driverFactory : driverFactories) {
        if (driverFactory.getSourceIds().contains(fragment.getPartitionedSource())) {
          checkState(
              partitionedDriverFactory == null, "multiple partitioned sources are not supported");
          partitionedDriverFactory = new DriverSplitRunnerFactory(driverFactory);
        } else {
          unpartitionedDriverFactories.add(new DriverSplitRunnerFactory(driverFactory));
        }
      }
      this.unpartitionedDriverFactories = unpartitionedDriverFactories.build();

      if (fragment.getDistribution() == PlanDistribution.SOURCE) {
        checkArgument(
            partitionedDriverFactory != null,
            "Fragment is partitioned, but no partitioned driver found");
      }
      this.partitionedSourceId = fragment.getPartitionedSource();
      this.partitionedDriverFactory = partitionedDriverFactory;
    }
  }
  private SqlTaskExecution(
      TaskStateMachine taskStateMachine,
      TaskContext taskContext,
      SharedBuffer sharedBuffer,
      PlanFragment fragment,
      LocalExecutionPlanner planner,
      TaskExecutor taskExecutor,
      QueryMonitor queryMonitor,
      Executor notificationExecutor) {
    this.taskStateMachine = checkNotNull(taskStateMachine, "taskStateMachine is null");
    this.taskId = taskStateMachine.getTaskId();
    this.taskContext = checkNotNull(taskContext, "taskContext is null");
    this.sharedBuffer = checkNotNull(sharedBuffer, "sharedBuffer is null");

    this.taskExecutor = checkNotNull(taskExecutor, "driverExecutor is null");
    this.notificationExecutor = checkNotNull(notificationExecutor, "notificationExecutor is null");

    this.queryMonitor = checkNotNull(queryMonitor, "queryMonitor is null");

    try (SetThreadName ignored = new SetThreadName("Task-%s", taskId)) {
      List<DriverFactory> driverFactories;
      try {
        OutputFactory outputOperatorFactory;
        if (fragment.getOutputPartitioning() == OutputPartitioning.NONE) {
          outputOperatorFactory = new TaskOutputFactory(sharedBuffer);
        } else if (fragment.getOutputPartitioning() == OutputPartitioning.HASH) {
          outputOperatorFactory = new PartitionedOutputFactory(sharedBuffer);
        } else {
          throw new PrestoException(
              NOT_SUPPORTED,
              format("OutputPartitioning %s is not supported", fragment.getOutputPartitioning()));
        }

        LocalExecutionPlan localExecutionPlan =
            planner.plan(
                taskContext.getSession(),
                fragment.getRoot(),
                fragment.getOutputLayout(),
                fragment.getSymbols(),
                fragment.getDistribution(),
                outputOperatorFactory);
        driverFactories = localExecutionPlan.getDriverFactories();
      } catch (Throwable e) {
        // planning failed
        taskStateMachine.failed(e);
        throw Throwables.propagate(e);
      }

      // index driver factories
      DriverSplitRunnerFactory partitionedDriverFactory = null;
      ImmutableList.Builder<DriverSplitRunnerFactory> unpartitionedDriverFactories =
          ImmutableList.builder();
      for (DriverFactory driverFactory : driverFactories) {
        if (driverFactory.getSourceIds().contains(fragment.getPartitionedSource())) {
          checkState(
              partitionedDriverFactory == null, "multiple partitioned sources are not supported");
          partitionedDriverFactory = new DriverSplitRunnerFactory(driverFactory);
        } else {
          unpartitionedDriverFactories.add(new DriverSplitRunnerFactory(driverFactory));
        }
      }
      this.unpartitionedDriverFactories = unpartitionedDriverFactories.build();

      if (fragment.getDistribution() == PlanDistribution.SOURCE) {
        checkArgument(
            partitionedDriverFactory != null,
            "Fragment is partitioned, but no partitioned driver found");
      }
      this.partitionedSourceId = fragment.getPartitionedSource();
      this.partitionedDriverFactory = partitionedDriverFactory;

      // don't register the task if it is already completed (most likely failed during planning
      // above)
      if (!taskStateMachine.getState().isDone()) {
        taskHandle = taskExecutor.addTask(taskId);
        taskStateMachine.addStateChangeListener(
            new RemoveTaskHandleWhenDone(taskExecutor, taskHandle));
      } else {
        taskHandle = null;
      }

      sharedBuffer.addStateChangeListener(
          new CheckTaskCompletionOnBufferFinish(SqlTaskExecution.this));
    }
  }
  private synchronized void updateTaskInfo(TaskInfo newValue, List<TaskSource> sources) {
    if (newValue.getState().isDone()) {
      // splits can be huge so clear the list
      pendingSplits.clear();
      fireSplitCountChanged(-pendingSourceSplitCount);
      pendingSourceSplitCount = 0;
    }

    int oldPartitionedSplitCount = getPartitionedSplitCount();

    // change to new value if old value is not changed and new value has a newer version
    AtomicBoolean workerRestarted = new AtomicBoolean();
    boolean updated =
        taskInfo.setIf(
            newValue,
            oldValue -> {
              // did the worker restart
              if (oldValue.getNodeInstanceId().isPresent()
                  && !oldValue.getNodeInstanceId().equals(newValue.getNodeInstanceId())) {
                workerRestarted.set(true);
                return false;
              }

              if (oldValue.getState().isDone()) {
                // never update if the task has reached a terminal state
                return false;
              }
              if (newValue.getVersion() < oldValue.getVersion()) {
                // don't update to an older version (same version is ok)
                return false;
              }
              return true;
            });

    if (workerRestarted.get()) {
      PrestoException exception =
          new PrestoException(
              WORKER_RESTARTED, format("%s (%s)", WORKER_RESTARTED_ERROR, newValue.getSelf()));
      failTask(exception);
      abort();
    }

    // remove acknowledged splits, which frees memory
    for (TaskSource source : sources) {
      PlanNodeId planNodeId = source.getPlanNodeId();
      int removed = 0;
      for (ScheduledSplit split : source.getSplits()) {
        if (pendingSplits.remove(planNodeId, split)) {
          removed++;
        }
      }
      if (planNodeId.equals(planFragment.getPartitionedSource())) {
        pendingSourceSplitCount -= removed;
      }
    }

    if (updated) {
      if (getTaskInfo().getState().isDone()) {
        fireSplitCountChanged(-oldPartitionedSplitCount);
      } else {
        fireSplitCountChanged(getPartitionedSplitCount() - oldPartitionedSplitCount);
      }
    }
  }
  public HttpRemoteTask(
      Session session,
      TaskId taskId,
      String nodeId,
      URI location,
      PlanFragment planFragment,
      Multimap<PlanNodeId, Split> initialSplits,
      OutputBuffers outputBuffers,
      HttpClient httpClient,
      Executor executor,
      ScheduledExecutorService errorScheduledExecutor,
      Duration minErrorDuration,
      Duration refreshMaxWait,
      JsonCodec<TaskInfo> taskInfoCodec,
      JsonCodec<TaskUpdateRequest> taskUpdateRequestCodec,
      SplitCountChangeListener splitCountChangeListener) {
    requireNonNull(session, "session is null");
    requireNonNull(taskId, "taskId is null");
    requireNonNull(nodeId, "nodeId is null");
    requireNonNull(location, "location is null");
    requireNonNull(planFragment, "planFragment1 is null");
    requireNonNull(outputBuffers, "outputBuffers is null");
    requireNonNull(httpClient, "httpClient is null");
    requireNonNull(executor, "executor is null");
    requireNonNull(taskInfoCodec, "taskInfoCodec is null");
    requireNonNull(taskUpdateRequestCodec, "taskUpdateRequestCodec is null");
    requireNonNull(splitCountChangeListener, "splitCountChangeListener is null");

    try (SetThreadName ignored = new SetThreadName("HttpRemoteTask-%s", taskId)) {
      this.taskId = taskId;
      this.session = session;
      this.nodeId = nodeId;
      this.planFragment = planFragment;
      this.outputBuffers.set(outputBuffers);
      this.httpClient = httpClient;
      this.executor = executor;
      this.errorScheduledExecutor = errorScheduledExecutor;
      this.taskInfoCodec = taskInfoCodec;
      this.taskUpdateRequestCodec = taskUpdateRequestCodec;
      this.updateErrorTracker =
          new RequestErrorTracker(
              taskId, location, minErrorDuration, errorScheduledExecutor, "updating task");
      this.getErrorTracker =
          new RequestErrorTracker(
              taskId, location, minErrorDuration, errorScheduledExecutor, "getting info for task");
      this.splitCountChangeListener = splitCountChangeListener;

      for (Entry<PlanNodeId, Split> entry :
          requireNonNull(initialSplits, "initialSplits is null").entries()) {
        ScheduledSplit scheduledSplit =
            new ScheduledSplit(nextSplitId.getAndIncrement(), entry.getValue());
        pendingSplits.put(entry.getKey(), scheduledSplit);
      }
      if (initialSplits.containsKey(planFragment.getPartitionedSource())) {
        pendingSourceSplitCount = initialSplits.get(planFragment.getPartitionedSource()).size();
        fireSplitCountChanged(pendingSourceSplitCount);
      }

      List<BufferInfo> bufferStates =
          outputBuffers
              .getBuffers()
              .keySet()
              .stream()
              .map(outputId -> new BufferInfo(outputId, false, 0, 0, PageBufferInfo.empty()))
              .collect(toImmutableList());

      TaskStats taskStats = new TaskStats(DateTime.now(), null);

      taskInfo =
          new StateMachine<>(
              "task " + taskId,
              executor,
              new TaskInfo(
                  taskId,
                  Optional.empty(),
                  TaskInfo.MIN_VERSION,
                  TaskState.PLANNED,
                  location,
                  DateTime.now(),
                  new SharedBufferInfo(BufferState.OPEN, true, true, 0, 0, 0, 0, bufferStates),
                  ImmutableSet.<PlanNodeId>of(),
                  taskStats,
                  ImmutableList.<ExecutionFailureInfo>of()));

      continuousTaskInfoFetcher = new ContinuousTaskInfoFetcher(refreshMaxWait);
    }
  }