@Override
 public final CommittedResult handleResult(
     CommittedBundle<?> inputBundle, TransformResult result) {
   CommittedResult committedResult = evaluationContext.handleResult(inputBundle, timers, result);
   for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
     allUpdates.offer(
         ExecutorUpdate.fromBundle(
             outputBundle, valueToConsumers.get(outputBundle.getPCollection())));
   }
   CommittedBundle<?> unprocessedInputs = committedResult.getUnprocessedInputs();
   if (unprocessedInputs != null && !Iterables.isEmpty(unprocessedInputs.getElements())) {
     if (inputBundle.getPCollection() == null) {
       // TODO: Split this logic out of an if statement
       pendingRootBundles.get(result.getTransform()).offer(unprocessedInputs);
     } else {
       allUpdates.offer(
           ExecutorUpdate.fromBundle(
               unprocessedInputs,
               Collections.<AppliedPTransform<?, ?, ?>>singleton(
                   committedResult.getTransform())));
     }
   }
   if (!committedResult.getProducedOutputTypes().isEmpty()) {
     state.set(ExecutorState.ACTIVE);
   }
   outstandingWork.decrementAndGet();
   return committedResult;
 }
  private <T> void evaluateBundle(
      final AppliedPTransform<?, ?, ?> transform,
      final CommittedBundle<T> bundle,
      final CompletionCallback onComplete) {
    TransformExecutorService transformExecutor;

    if (isKeyed(bundle.getPCollection())) {
      final StepAndKey stepAndKey = StepAndKey.of(transform, bundle.getKey());
      // This executor will remain reachable until it has executed all scheduled transforms.
      // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
      // a reference to the scheduled TransformExecutor callable. Follow-up TransformExecutors
      // (scheduled due to the completion of another TransformExecutor) are provided to the
      // ExecutorService before the Earlier TransformExecutor callable completes.
      transformExecutor = executorServices.getUnchecked(stepAndKey);
    } else {
      transformExecutor = parallelExecutorService;
    }

    Collection<ModelEnforcementFactory> enforcements =
        MoreObjects.firstNonNull(
            transformEnforcements.get(transform.getTransform().getClass()),
            Collections.<ModelEnforcementFactory>emptyList());

    TransformExecutor<T> callable =
        TransformExecutor.create(
            evaluationContext,
            registry,
            enforcements,
            bundle,
            transform,
            onComplete,
            transformExecutor);
    outstandingWork.incrementAndGet();
    transformExecutor.schedule(callable);
  }