You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by pa...@apache.org on 2019/05/20 22:14:48 UTC

[beam] branch master updated: [BEAM-6880] Remove deprecated Reference Runner code. (#8380)

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

pabloem pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new eddc83a  [BEAM-6880] Remove deprecated Reference Runner code. (#8380)
eddc83a is described below

commit eddc83a33e74a606b0584eda75e4c2257e666032
Author: Daniel Oliveira <da...@gmail.com>
AuthorDate: Mon May 20 15:14:35 2019 -0700

    [BEAM-6880] Remove deprecated Reference Runner code. (#8380)
    
    * [BEAM-6880] Remove Java Reference Runner test code.
    
    * [BEAM-6880] Remove additional mentions of removed code.
---
 runners/direct-java/build.gradle                   |  43 --
 .../runners/direct/portable/BundleFactory.java     |  47 --
 .../BundleFactoryOutputReceiverFactory.java        |  68 ---
 .../runners/direct/portable/BundleProcessor.java   |  33 --
 .../runners/direct/portable/CommittedBundle.java   |  86 ---
 .../runners/direct/portable/CommittedResult.java   |  63 ---
 .../direct/portable/CompletionCallback.java        |  42 --
 .../CopyOnAccessInMemoryStateInternals.java        | 463 ----------------
 .../runners/direct/portable/DirectGroupByKey.java  | 133 -----
 .../runners/direct/portable/DirectMetrics.java     | 307 -----------
 .../direct/portable/DirectStateAndTimers.java      |  70 ---
 .../direct/portable/DirectTimerInternals.java      | 105 ----
 .../direct/portable/DirectTransformExecutor.java   | 151 ------
 .../direct/portable/DisplayDataValidator.java      |  67 ---
 .../direct/portable/EmptyInputProvider.java        |  38 --
 .../runners/direct/portable/EvaluationContext.java | 333 ------------
 ...valuationContextStepStateAndTimersProvider.java |  39 --
 .../direct/portable/ExecutorServiceFactory.java    |  30 --
 .../portable/ExecutorServiceParallelExecutor.java  | 392 --------------
 .../direct/portable/FlattenEvaluatorFactory.java   |  79 ---
 .../GroupAlsoByWindowEvaluatorFactory.java         | 282 ----------
 .../portable/GroupByKeyOnlyEvaluatorFactory.java   | 157 ------
 .../portable/ImmutableListBundleFactory.java       | 171 ------
 .../direct/portable/ImpulseEvaluatorFactory.java   | 110 ----
 .../direct/portable/PCollectionViewWindow.java     |  66 ---
 .../direct/portable/PCollectionViewWriter.java     |  34 --
 .../portable/PassthroughTransformEvaluator.java    |  47 --
 .../runners/direct/portable/PipelineExecutor.java  |  61 ---
 .../runners/direct/portable/PortableGraph.java     |  73 ---
 .../runners/direct/portable/QuiescenceDriver.java  | 328 ------------
 .../runners/direct/portable/ReferenceRunner.java   | 559 -------------------
 .../portable/RemoteStageEvaluatorFactory.java      |  95 ----
 .../runners/direct/portable/RootInputProvider.java |  44 --
 .../direct/portable/RootProviderRegistry.java      |  68 ---
 .../beam/runners/direct/portable/SourceShard.java  |  33 --
 .../SplittableRemoteStageEvaluatorFactory.java     | 174 ------
 .../beam/runners/direct/portable/StepAndKey.java   |  67 ---
 .../direct/portable/StepStateAndTimers.java        |  34 --
 .../direct/portable/StepTransformResult.java       | 135 -----
 .../direct/portable/TransformEvaluator.java        |  45 --
 .../direct/portable/TransformEvaluatorFactory.java |  58 --
 .../portable/TransformEvaluatorRegistry.java       | 126 -----
 .../runners/direct/portable/TransformExecutor.java |  21 -
 .../direct/portable/TransformExecutorFactory.java  |  29 -
 .../direct/portable/TransformExecutorService.java  |  38 --
 .../direct/portable/TransformExecutorServices.java | 180 -------
 .../runners/direct/portable/TransformResult.java   | 101 ----
 .../runners/direct/portable/UncommittedBundle.java |  56 --
 .../direct/portable/WatermarkCallbackExecutor.java | 171 ------
 .../direct/portable/WindowEvaluatorFactory.java    | 123 -----
 .../artifact/LocalArtifactStagingLocation.java     | 116 ----
 .../LocalFileSystemArtifactRetrievalService.java   | 119 -----
 .../LocalFileSystemArtifactStagerService.java      | 282 ----------
 .../UnsupportedArtifactRetrievalService.java       |  42 --
 .../direct/portable/artifact/package-info.java     |  20 -
 .../runners/direct/portable/job/PreparingJob.java  |  59 --
 .../portable/job/ReferenceRunnerJobServer.java     | 164 ------
 .../portable/job/ReferenceRunnerJobService.java    | 308 -----------
 .../runners/direct/portable/job/package-info.java  |  23 -
 .../beam/runners/direct/portable/package-info.java |  25 -
 .../runners/direct/DirectRunnerApiSurfaceTest.java |   2 -
 .../BundleFactoryOutputReceiverFactoryTest.java    | 262 ---------
 .../direct/portable/CommittedResultTest.java       | 129 -----
 .../CopyOnAccessInMemoryStateInternalsTest.java    | 591 ---------------------
 .../runners/direct/portable/DirectMetricsTest.java | 232 --------
 .../direct/portable/DirectTimerInternalsTest.java  | 132 -----
 .../portable/DirectTransformExecutorTest.java      | 330 ------------
 .../direct/portable/EvaluationContextTest.java     | 315 -----------
 .../direct/portable/ExecutableGraphBuilder.java    |  81 ---
 .../portable/FlattenEvaluatorFactoryTest.java      | 122 -----
 .../GroupByKeyOnlyEvaluatorFactoryTest.java        | 247 ---------
 .../portable/ImmutableListBundleFactoryTest.java   | 234 --------
 .../portable/ImpulseEvaluatorFactoryTest.java      | 130 -----
 .../beam/runners/direct/portable/MockClock.java    |  65 ---
 .../runners/direct/portable/PortableGraphTest.java | 176 ------
 .../direct/portable/ReferenceRunnerTest.java       | 213 --------
 .../portable/RemoteStageEvaluatorFactoryTest.java  | 238 ---------
 .../direct/portable/StepTransformResultTest.java   |  95 ----
 .../portable/TransformExecutorServicesTest.java    | 143 -----
 .../portable/WatermarkCallbackExecutorTest.java    | 136 -----
 .../portable/WindowEvaluatorFactoryTest.java       | 314 -----------
 .../artifact/LocalArtifactStagingLocationTest.java | 122 -----
 ...ocalFileSystemArtifactRetrievalServiceTest.java | 240 ---------
 .../LocalFileSystemArtifactStagerServiceTest.java  | 313 -----------
 .../UnsupportedArtifactRetrievalServiceTest.java   | 150 ------
 .../job/ReferenceRunnerJobServiceTest.java         | 150 ------
 runners/reference/job-server/build.gradle          |  54 --
 .../src/main/resources/beam/spotbugs-filter.xml    |  19 -
 settings.gradle                                    |   1 -
 89 files changed, 12469 deletions(-)

diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle
index 328de71..e06194c 100644
--- a/runners/direct-java/build.gradle
+++ b/runners/direct-java/build.gradle
@@ -54,7 +54,6 @@ evaluationDependsOn(":sdks:java:core")
 configurations {
   needsRunner
   validatesRunner
-  validatesPortableRunner
 }
 
 dependencies {
@@ -70,7 +69,6 @@ dependencies {
   shadow library.java.args4j
   provided library.java.hamcrest_core
   provided library.java.junit
-  testRuntime project(path: ":sdks:java:harness")
   shadowTest project(path: ":sdks:java:core", configuration: "shadowTest")
   shadowTest project(path: ":runners:core-java", configuration: "shadowTest")
   shadowTest library.java.guava_testlib
@@ -90,13 +88,6 @@ dependencies {
   validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest")
   validatesRunner project(path: project.path, configuration: "shadow")
   validatesRunner project(path: project.path, configuration: "shadowTest")
-  validatesPortableRunner project(path: ":runners:core-construction-java", configuration: "shadowTest")
-  validatesPortableRunner project(path: ":runners:core-java", configuration: "shadowTest")
-  validatesPortableRunner project(path: ":runners:java-fn-execution", configuration: "shadowTest")
-  validatesPortableRunner project(path: ":runners:reference:java", configuration: "shadowTest")
-  validatesPortableRunner project(path: ":sdks:java:core", configuration: "shadowTest")
-  validatesPortableRunner project(path: project.path, configuration: "shadow")
-  validatesPortableRunner project(path: project.path, configuration: "shadowTest")
 }
 
 task needsRunnerTests(type: Test) {
@@ -160,37 +151,3 @@ createJavaExamplesArchetypeValidationTask(type: 'MobileGaming',
   gcsBucket: gcsBucket,
   bqDataset: bqDataset,
   pubsubTopic: pubsubTopic)
-
-createPortableValidatesRunnerTask(
-        jobServerDriver: "org.apache.beam.runners.direct.portable.job.ReferenceRunnerJobServer",
-        testClasspathConfiguration: configurations.validatesPortableRunner,
-        jobServerConfig: "--port=0",
-        testCategories: {
-          includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
-          excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
-          excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesCrossLanguageTransforms'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesDistributionMetrics'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
-          // TODO(BEAM-5452): Support metrics.
-          excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesAttemptedMetrics'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesCounterMetrics'
-          // TODO(BEAM-3743): Support SplittableDoFn
-          excludeCategories 'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
-          // TODO(BEAM-2928): Support sideinput.
-          excludeCategories 'org.apache.beam.sdk.testing.UsesSideInputs'
-          // TODO(BEAM-2917): Support user state.
-          excludeCategories 'org.apache.beam.sdk.testing.UsesStatefulParDo'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
-          excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
-          // TODO(BEAM-4680): Support user timers.
-          excludeCategories 'org.apache.beam.sdk.testing.UsesTimersInParDo'
-        },
-)
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactory.java
deleted file mode 100644
index 118892c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-
-/** A factory that creates {@link UncommittedBundle UncommittedBundles}. */
-interface BundleFactory {
-  /**
-   * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle do not
-   * belong to a {@link PCollection}.
-   *
-   * <p>For use in creating inputs to root transforms.
-   */
-  <T> UncommittedBundle<T> createRootBundle();
-
-  /**
-   * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
-   * belong to the {@code output} {@link PCollection}.
-   */
-  <T> UncommittedBundle<T> createBundle(PCollectionNode output);
-
-  /**
-   * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
-   * {@code DirectGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
-   * belong to the {@code output} {@link PCollection}.
-   */
-  <K, T> UncommittedBundle<T> createKeyedBundle(StructuralKey<K> key, PCollectionNode output);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactory.java
deleted file mode 100644
index 53c2cb9..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactory.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.function.Consumer;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An {@link OutputReceiverFactory} which adds received elements to {@link UncommittedBundle}
- * instances. The produced {@link UncommittedBundle bundles} are added to a provided {@link
- * StepTransformResult.Builder StepTransformResult Builder}.
- */
-class BundleFactoryOutputReceiverFactory implements OutputReceiverFactory {
-  private final BundleFactory bundleFactory;
-  private final RunnerApi.Components components;
-
-  private final Consumer<UncommittedBundle<?>> bundleConsumer;
-
-  private BundleFactoryOutputReceiverFactory(
-      BundleFactory bundleFactory,
-      Components components,
-      Consumer<UncommittedBundle<?>> bundleConsumer) {
-    this.bundleFactory = bundleFactory;
-    this.components = components;
-    this.bundleConsumer = bundleConsumer;
-  }
-
-  public static OutputReceiverFactory create(
-      BundleFactory bundleFactory,
-      Components components,
-      Consumer<UncommittedBundle<?>> resultBuilder) {
-    return new BundleFactoryOutputReceiverFactory(bundleFactory, components, resultBuilder);
-  }
-
-  @Override
-  public <OutputT> FnDataReceiver<OutputT> create(String pCollectionId) {
-    PCollectionNode pcollection =
-        PipelineNode.pCollection(pCollectionId, components.getPcollectionsOrThrow(pCollectionId));
-    return create(pcollection);
-  }
-
-  private <ElemT, OutputT> FnDataReceiver<OutputT> create(PCollectionNode pcollection) {
-    UncommittedBundle<ElemT> bundle = bundleFactory.createBundle(pcollection);
-    bundleConsumer.accept(bundle);
-    return input -> bundle.add((WindowedValue<ElemT>) input);
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleProcessor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleProcessor.java
deleted file mode 100644
index a08e1b8..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleProcessor.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.local.Bundle;
-
-/**
- * An executor that is capable of processing some bundle of input over some executable stage or
- * step.
- */
-interface BundleProcessor<
-    CollectionT, BundleT extends Bundle<?, ? extends CollectionT>, ExecutableT> {
-  /**
-   * Execute the provided bundle using the provided Executable, calling back to the {@link
-   * CompletionCallback} when execution completes.
-   */
-  void process(BundleT bundle, ExecutableT consumer, CompletionCallback onComplete);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedBundle.java
deleted file mode 100644
index 93926d6..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedBundle.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.local.Bundle;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-
-/**
- * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
- * eventually committed. Committed elements are executed by the {@link PTransform PTransforms} that
- * consume the {@link PCollection} this bundle is a part of at a later point.
- *
- * @param <T> the type of elements contained within this bundle
- */
-interface CommittedBundle<T> extends Bundle<T, PCollectionNode> {
-  /** Returns the PCollection that the elements of this bundle belong to. */
-  @Nullable
-  @Override
-  PCollectionNode getPCollection();
-
-  /**
-   * Returns the key that was output in the most recent {@code GroupByKey} in the execution of this
-   * bundle.
-   */
-  @Override
-  StructuralKey<?> getKey();
-
-  /**
-   * Returns an {@link Iterable} containing all of the elements that have been added to this {@link
-   * CommittedBundle}.
-   */
-  Iterable<WindowedValue<T>> getElements();
-
-  /**
-   * Return the minimum timestamp among elements in this bundle.
-   *
-   * <p>This should be equivalent to iterating over all of the elements within a bundle and
-   * selecting the minimum timestamp from among them.
-   */
-  @Override
-  Instant getMinimumTimestamp();
-
-  /**
-   * Returns the processing time output watermark at the time the producing {@code Executable}
-   * committed this bundle. Downstream synchronized processing time watermarks cannot progress past
-   * this point before consuming this bundle.
-   *
-   * <p>This value is no greater than the earliest incomplete processing time or synchronized
-   * processing time {@link TimerData timer} at the time this bundle was committed, including any
-   * timers that fired to produce this bundle.
-   */
-  @Override
-  Instant getSynchronizedProcessingOutputWatermark();
-  /**
-   * Return a new {@link CommittedBundle} that is like this one, except calls to {@link
-   * #getElements()} will return the provided elements. This bundle is unchanged.
-   *
-   * <p>The value of the {@link #getSynchronizedProcessingOutputWatermark() synchronized processing
-   * output watermark} of the returned {@link CommittedBundle} is equal to the value returned from
-   * the current bundle. This is used to ensure a {@link PTransform} that could not complete
-   * processing on input elements properly holds the synchronized processing time to the appropriate
-   * value.
-   */
-  CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedResult.java
deleted file mode 100644
index 5487118..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedResult.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import com.google.auto.value.AutoValue;
-import java.util.Set;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-
-/** A {@link TransformResult} that has been committed. */
-@AutoValue
-abstract class CommittedResult<ExecutableT> {
-  /** Returns the {@link PTransformNode} that produced this result. */
-  public abstract ExecutableT getExecutable();
-
-  /**
-   * Returns the {@link CommittedBundle} that contains the input elements that could not be
-   * processed by the evaluation. The returned optional is present if there were any unprocessed
-   * input elements, and absent otherwise.
-   */
-  public abstract Optional<? extends CommittedBundle<?>> getUnprocessedInputs();
-  /** Returns the outputs produced by the transform. */
-  public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
-
-  /**
-   * Returns if the transform that produced this result produced outputs.
-   *
-   * <p>Transforms that produce output via modifying the state of the runner (e.g. {@link
-   * CreatePCollectionView}) should explicitly set this to true. If {@link #getOutputs()} returns a
-   * nonempty iterable, this will also return true.
-   */
-  public abstract Set<OutputType> getProducedOutputTypes();
-
-  public static CommittedResult<PTransformNode> create(
-      TransformResult<?> original,
-      Optional<? extends CommittedBundle<?>> unprocessedElements,
-      Iterable<? extends CommittedBundle<?>> outputs,
-      Set<OutputType> producedOutputs) {
-    return new AutoValue_CommittedResult<>(
-        original.getTransform(), unprocessedElements, outputs, producedOutputs);
-  }
-
-  enum OutputType {
-    PCOLLECTION_VIEW,
-    BUNDLE
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CompletionCallback.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CompletionCallback.java
deleted file mode 100644
index 4d69044..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CompletionCallback.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-
-/** A callback for completing a bundle of input. */
-interface CompletionCallback {
-  /** Handle a successful result, returning the committed outputs of the result. */
-  CommittedResult handleResult(CommittedBundle<?> inputBundle, TransformResult<?> result);
-
-  /**
-   * Handle an input bundle that did not require processing.
-   *
-   * <p>This occurs when a Source has no splits that can currently produce outputs.
-   */
-  void handleEmpty(PTransformNode transform);
-
-  /** Handle a result that terminated abnormally due to the provided {@link Exception}. */
-  void handleException(CommittedBundle<?> inputBundle, Exception t);
-
-  /**
-   * Handle a result that terminated abnormally due to the provided {@link Error}. The pipeline
-   * should be shut down, and the Error propagated.
-   */
-  void handleError(Error err);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternals.java
deleted file mode 100644
index d217121..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternals.java
+++ /dev/null
@@ -1,463 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryBag;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryCombiningState;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryMap;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemorySet;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryState;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryStateBinder;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryValue;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryWatermarkHold;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateTable;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTag.StateBinder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.CombiningState;
-import org.apache.beam.sdk.state.MapState;
-import org.apache.beam.sdk.state.SetState;
-import org.apache.beam.sdk.state.State;
-import org.apache.beam.sdk.state.StateContext;
-import org.apache.beam.sdk.state.StateContexts;
-import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.state.WatermarkHoldState;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
-import org.apache.beam.sdk.util.CombineFnUtil;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.joda.time.Instant;
-
-/**
- * {@link StateInternals} built on top of an underlying {@link StateTable} that contains instances
- * of {@link InMemoryState}. Whenever state that exists in the underlying {@link StateTable} is
- * accessed, an independent copy will be created within this table.
- */
-class CopyOnAccessInMemoryStateInternals<K> implements StateInternals {
-  private final CopyOnAccessInMemoryStateTable table;
-
-  private K key;
-
-  /**
-   * Creates a new {@link CopyOnAccessInMemoryStateInternals} with the underlying (possibly null)
-   * StateInternals.
-   */
-  public static <K> CopyOnAccessInMemoryStateInternals withUnderlying(
-      K key, @Nullable CopyOnAccessInMemoryStateInternals underlying) {
-    return new CopyOnAccessInMemoryStateInternals<>(key, underlying);
-  }
-
-  private CopyOnAccessInMemoryStateInternals(K key, CopyOnAccessInMemoryStateInternals underlying) {
-    this.key = key;
-    table = new CopyOnAccessInMemoryStateTable(underlying == null ? null : underlying.table);
-  }
-
-  /**
-   * Ensures this {@link CopyOnAccessInMemoryStateInternals} is complete. Other copies of state for
-   * the same Step and Key may be discarded after invoking this method.
-   *
-   * <p>For each {@link StateNamespace}, for each {@link StateTag address} in that namespace that
-   * has not been bound in this {@link CopyOnAccessInMemoryStateInternals}, put a reference to that
-   * state within this {@link StateInternals}.
-   *
-   * <p>Additionally, stores the {@link WatermarkHoldState} with the earliest time bound in the
-   * state table after the commit is completed, enabling calls to {@link
-   * #getEarliestWatermarkHold()}.
-   *
-   * @return this table
-   */
-  public CopyOnAccessInMemoryStateInternals<K> commit() {
-    table.commit();
-    return this;
-  }
-
-  /**
-   * Gets the earliest Watermark Hold present in this table.
-   *
-   * <p>Must be called after this state has been committed. Will throw an {@link
-   * IllegalStateException} if the state has not been committed.
-   */
-  public Instant getEarliestWatermarkHold() {
-    // After commit, the watermark hold is always present, but may be
-    // BoundedWindow#TIMESTAMP_MAX_VALUE if there is no hold set.
-    checkState(
-        table.earliestWatermarkHold.isPresent(),
-        "Can't get the earliest watermark hold in a %s before it is committed",
-        getClass().getSimpleName());
-    return table.earliestWatermarkHold.get();
-  }
-
-  @Override
-  public <T extends State> T state(
-      StateNamespace namespace, StateTag<T> address, StateContext<?> c) {
-    return table.get(namespace, address, c);
-  }
-
-  @Override
-  public Object getKey() {
-    return key;
-  }
-
-  public boolean isEmpty() {
-    return Iterables.isEmpty(table.values());
-  }
-
-  /**
-   * A {@link StateTable} that, when a value is retrieved with {@link StateTable#get(StateNamespace,
-   * StateTag, StateContext)}, first attempts to obtain a copy of existing {@link State} from an
-   * underlying {@link StateTable}.
-   */
-  private static class CopyOnAccessInMemoryStateTable extends StateTable {
-    private Optional<StateTable> underlying;
-
-    /**
-     * The StateBinderFactory currently in use by this {@link CopyOnAccessInMemoryStateTable}.
-     *
-     * <p>There are three {@link StateBinderFactory} implementations used by the {@link
-     * CopyOnAccessInMemoryStateTable}.
-     *
-     * <ul>
-     *   <li>The default {@link StateBinderFactory} is a {@link CopyOnBindBinderFactory}, allowing
-     *       the table to copy any existing {@link State} values to this {@link StateTable} from the
-     *       underlying table when accessed, at which point mutations will not be visible to the
-     *       underlying table - effectively a "Copy by Value" binder.
-     *   <li>During the execution of the {@link #commit()} method, this is a {@link
-     *       ReadThroughBinderFactory}, which copies the references to the existing {@link State}
-     *       objects to this {@link StateTable}.
-     *   <li>After the execution of the {@link #commit()} method, this is an instance of {@link
-     *       InMemoryStateBinderFactory}, which constructs new instances of state when a {@link
-     *       StateTag} is bound.
-     * </ul>
-     */
-    private StateBinderFactory binderFactory;
-
-    /** The earliest watermark hold in this table. */
-    private Optional<Instant> earliestWatermarkHold;
-
-    public CopyOnAccessInMemoryStateTable(StateTable underlying) {
-      this.underlying = Optional.fromNullable(underlying);
-      binderFactory = new CopyOnBindBinderFactory(this.underlying);
-      earliestWatermarkHold = Optional.absent();
-    }
-
-    /**
-     * Copies all values in the underlying table to this table, then discards the underlying table.
-     *
-     * <p>If there is an underlying table, this replaces the existing {@link
-     * CopyOnBindBinderFactory} with a {@link ReadThroughBinderFactory}, then reads all of the
-     * values in the existing table, binding the state values to this table. The old StateTable
-     * should be discarded after the call to {@link #commit()}.
-     *
-     * <p>After copying all of the existing values, replace the binder factory with an instance of
-     * {@link InMemoryStateBinderFactory} to construct new values, since all existing values are
-     * bound in this {@link StateTable table} and this table represents the canonical state.
-     */
-    private void commit() {
-      Instant earliestHold = getEarliestWatermarkHold();
-      if (underlying.isPresent()) {
-        ReadThroughBinderFactory readThroughBinder =
-            new ReadThroughBinderFactory<>(underlying.get());
-        binderFactory = readThroughBinder;
-        Instant earliestUnderlyingHold = readThroughBinder.readThroughAndGetEarliestHold(this);
-        if (earliestUnderlyingHold.isBefore(earliestHold)) {
-          earliestHold = earliestUnderlyingHold;
-        }
-      }
-      earliestWatermarkHold = Optional.of(earliestHold);
-      clearEmpty();
-      binderFactory = new InMemoryStateBinderFactory();
-      underlying = Optional.absent();
-    }
-
-    /**
-     * Get the earliest watermark hold in this table. Ignores the contents of any underlying table.
-     */
-    private Instant getEarliestWatermarkHold() {
-      Instant earliest = BoundedWindow.TIMESTAMP_MAX_VALUE;
-      for (State existingState : this.values()) {
-        if (existingState instanceof WatermarkHoldState) {
-          Instant hold = ((WatermarkHoldState) existingState).read();
-          if (hold != null && hold.isBefore(earliest)) {
-            earliest = hold;
-          }
-        }
-      }
-      return earliest;
-    }
-
-    /**
-     * Clear all empty {@link StateNamespace StateNamespaces} from this table. If all states are
-     * empty, clear the entire table.
-     *
-     * <p>Because {@link InMemoryState} is not removed from the {@link StateTable} after it is
-     * cleared, in case contents are modified after being cleared, the table must be explicitly
-     * checked to ensure that it contains state and removed if not (otherwise we may never use the
-     * table again).
-     */
-    private void clearEmpty() {
-      Collection<StateNamespace> emptyNamespaces = new HashSet<>(this.getNamespacesInUse());
-      for (StateNamespace namespace : this.getNamespacesInUse()) {
-        for (State existingState : this.getTagsInUse(namespace).values()) {
-          if (!((InMemoryState<?>) existingState).isCleared()) {
-            emptyNamespaces.remove(namespace);
-            break;
-          }
-        }
-      }
-      for (StateNamespace empty : emptyNamespaces) {
-        this.clearNamespace(empty);
-      }
-    }
-
-    @Override
-    protected StateBinder binderForNamespace(final StateNamespace namespace, StateContext<?> c) {
-      return binderFactory.forNamespace(namespace, c);
-    }
-
-    private interface StateBinderFactory {
-      StateBinder forNamespace(StateNamespace namespace, StateContext<?> c);
-    }
-
-    /**
-     * {@link StateBinderFactory} that creates a copy of any existing state when the state is bound.
-     */
-    private static class CopyOnBindBinderFactory implements StateBinderFactory {
-      private final Optional<StateTable> underlying;
-
-      public CopyOnBindBinderFactory(Optional<StateTable> underlying) {
-        this.underlying = underlying;
-      }
-
-      private boolean containedInUnderlying(StateNamespace namespace, StateTag<?> tag) {
-        return underlying.isPresent()
-            && underlying.get().isNamespaceInUse(namespace)
-            && underlying.get().getTagsInUse(namespace).containsKey(tag);
-      }
-
-      @Override
-      public StateBinder forNamespace(final StateNamespace namespace, final StateContext<?> c) {
-        return new StateBinder() {
-          @Override
-          public WatermarkHoldState bindWatermark(
-              StateTag<WatermarkHoldState> address, TimestampCombiner timestampCombiner) {
-            if (containedInUnderlying(namespace, address)) {
-              @SuppressWarnings("unchecked")
-              InMemoryState<? extends WatermarkHoldState> existingState =
-                  (InMemoryState<? extends WatermarkHoldState>)
-                      underlying.get().get(namespace, address, c);
-              return existingState.copy();
-            } else {
-              return new InMemoryWatermarkHold<>(timestampCombiner);
-            }
-          }
-
-          @Override
-          public <T> ValueState<T> bindValue(StateTag<ValueState<T>> address, Coder<T> coder) {
-            if (containedInUnderlying(namespace, address)) {
-              @SuppressWarnings("unchecked")
-              InMemoryState<? extends ValueState<T>> existingState =
-                  (InMemoryState<? extends ValueState<T>>)
-                      underlying.get().get(namespace, address, c);
-              return existingState.copy();
-            } else {
-              return new InMemoryValue<>(coder);
-            }
-          }
-
-          @Override
-          public <InputT, AccumT, OutputT>
-              CombiningState<InputT, AccumT, OutputT> bindCombiningValue(
-                  StateTag<CombiningState<InputT, AccumT, OutputT>> address,
-                  Coder<AccumT> accumCoder,
-                  CombineFn<InputT, AccumT, OutputT> combineFn) {
-            if (containedInUnderlying(namespace, address)) {
-              @SuppressWarnings("unchecked")
-              InMemoryState<? extends CombiningState<InputT, AccumT, OutputT>> existingState =
-                  (InMemoryState<? extends CombiningState<InputT, AccumT, OutputT>>)
-                      underlying.get().get(namespace, address, c);
-              return existingState.copy();
-            } else {
-              return new InMemoryCombiningState<>(combineFn, accumCoder);
-            }
-          }
-
-          @Override
-          public <T> BagState<T> bindBag(StateTag<BagState<T>> address, Coder<T> elemCoder) {
-            if (containedInUnderlying(namespace, address)) {
-              @SuppressWarnings("unchecked")
-              InMemoryState<? extends BagState<T>> existingState =
-                  (InMemoryState<? extends BagState<T>>)
-                      underlying.get().get(namespace, address, c);
-              return existingState.copy();
-            } else {
-              return new InMemoryBag<>(elemCoder);
-            }
-          }
-
-          @Override
-          public <T> SetState<T> bindSet(StateTag<SetState<T>> address, Coder<T> elemCoder) {
-            if (containedInUnderlying(namespace, address)) {
-              @SuppressWarnings("unchecked")
-              InMemoryState<? extends SetState<T>> existingState =
-                  (InMemoryState<? extends SetState<T>>)
-                      underlying.get().get(namespace, address, c);
-              return existingState.copy();
-            } else {
-              return new InMemorySet<>(elemCoder);
-            }
-          }
-
-          @Override
-          public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
-              StateTag<MapState<KeyT, ValueT>> address,
-              Coder<KeyT> mapKeyCoder,
-              Coder<ValueT> mapValueCoder) {
-            if (containedInUnderlying(namespace, address)) {
-              @SuppressWarnings("unchecked")
-              InMemoryState<? extends MapState<KeyT, ValueT>> existingState =
-                  (InMemoryState<? extends MapState<KeyT, ValueT>>)
-                      underlying.get().get(namespace, address, c);
-              return existingState.copy();
-            } else {
-              return new InMemoryMap<>(mapKeyCoder, mapValueCoder);
-            }
-          }
-
-          @Override
-          public <InputT, AccumT, OutputT>
-              CombiningState<InputT, AccumT, OutputT> bindCombiningValueWithContext(
-                  StateTag<CombiningState<InputT, AccumT, OutputT>> address,
-                  Coder<AccumT> accumCoder,
-                  CombineFnWithContext<InputT, AccumT, OutputT> combineFn) {
-            return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c));
-          }
-        };
-      }
-    }
-
-    /**
-     * {@link StateBinderFactory} that reads directly from the underlying table. Used during calls
-     * to {@link CopyOnAccessInMemoryStateTable#commit()} to read all values from the underlying
-     * table.
-     */
-    private static class ReadThroughBinderFactory<K> implements StateBinderFactory {
-      private final StateTable underlying;
-
-      public ReadThroughBinderFactory(StateTable underlying) {
-        this.underlying = underlying;
-      }
-
-      public Instant readThroughAndGetEarliestHold(StateTable readTo) {
-        Instant earliestHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
-        for (StateNamespace namespace : underlying.getNamespacesInUse()) {
-          for (Map.Entry<StateTag, State> existingState :
-              underlying.getTagsInUse(namespace).entrySet()) {
-            if (!((InMemoryState<?>) existingState.getValue()).isCleared()) {
-              // Only read through non-cleared values to ensure that completed windows are
-              // eventually discarded, and remember the earliest watermark hold from among those
-              // values.
-              State state =
-                  readTo.get(namespace, existingState.getKey(), StateContexts.nullContext());
-              if (state instanceof WatermarkHoldState) {
-                Instant hold = ((WatermarkHoldState) state).read();
-                if (hold != null && hold.isBefore(earliestHold)) {
-                  earliestHold = hold;
-                }
-              }
-            }
-          }
-        }
-        return earliestHold;
-      }
-
-      @Override
-      public StateBinder forNamespace(final StateNamespace namespace, final StateContext<?> c) {
-        return new StateBinder() {
-          @Override
-          public WatermarkHoldState bindWatermark(
-              StateTag<WatermarkHoldState> address, TimestampCombiner timestampCombiner) {
-            return underlying.get(namespace, address, c);
-          }
-
-          @Override
-          public <T> ValueState<T> bindValue(StateTag<ValueState<T>> address, Coder<T> coder) {
-            return underlying.get(namespace, address, c);
-          }
-
-          @Override
-          public <InputT, AccumT, OutputT>
-              CombiningState<InputT, AccumT, OutputT> bindCombiningValue(
-                  StateTag<CombiningState<InputT, AccumT, OutputT>> address,
-                  Coder<AccumT> accumCoder,
-                  CombineFn<InputT, AccumT, OutputT> combineFn) {
-            return underlying.get(namespace, address, c);
-          }
-
-          @Override
-          public <T> BagState<T> bindBag(StateTag<BagState<T>> address, Coder<T> elemCoder) {
-            return underlying.get(namespace, address, c);
-          }
-
-          @Override
-          public <T> SetState<T> bindSet(StateTag<SetState<T>> address, Coder<T> elemCoder) {
-            return underlying.get(namespace, address, c);
-          }
-
-          @Override
-          public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
-              StateTag<MapState<KeyT, ValueT>> address,
-              Coder<KeyT> mapKeyCoder,
-              Coder<ValueT> mapValueCoder) {
-            return underlying.get(namespace, address, c);
-          }
-
-          @Override
-          public <InputT, AccumT, OutputT>
-              CombiningState<InputT, AccumT, OutputT> bindCombiningValueWithContext(
-                  StateTag<CombiningState<InputT, AccumT, OutputT>> address,
-                  Coder<AccumT> accumCoder,
-                  CombineFnWithContext<InputT, AccumT, OutputT> combineFn) {
-            return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c));
-          }
-        };
-      }
-    }
-
-    private static class InMemoryStateBinderFactory implements StateBinderFactory {
-
-      public InMemoryStateBinderFactory() {}
-
-      @Override
-      public StateBinder forNamespace(StateNamespace namespace, StateContext<?> c) {
-        return new InMemoryStateBinder(c);
-      }
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectGroupByKey.java
deleted file mode 100644
index 37b5430..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectGroupByKey.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItemCoder;
-import org.apache.beam.runners.core.construction.ForwardingPTransform;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.WindowingStrategy;
-
-class DirectGroupByKey<K, V>
-    extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-  private final PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> original;
-
-  static final String DIRECT_GBKO_URN = "urn:beam:directrunner:transforms:gbko:v1";
-  static final String DIRECT_GABW_URN = "urn:beam:directrunner:transforms:gabw:v1";
-  private final WindowingStrategy<?, ?> outputWindowingStrategy;
-
-  DirectGroupByKey(
-      PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> original,
-      WindowingStrategy<?, ?> outputWindowingStrategy) {
-    this.original = original;
-    this.outputWindowingStrategy = outputWindowingStrategy;
-  }
-
-  @Override
-  public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
-    return original;
-  }
-
-  @Override
-  public PCollection<KV<K, Iterable<V>>> expand(PCollection<KV<K, V>> input) {
-    // This operation groups by the combination of key and window,
-    // merging windows as needed, using the windows assigned to the
-    // key/value input elements and the window merge operation of the
-    // window function associated with the input PCollection.
-    WindowingStrategy<?, ?> inputWindowingStrategy = input.getWindowingStrategy();
-
-    // By default, implement GroupByKey via a series of lower-level operations.
-    return input
-        .apply(new DirectGroupByKeyOnly<>())
-
-        // Group each key's values by window, merging windows as needed.
-        .apply(
-            "GroupAlsoByWindow",
-            new DirectGroupAlsoByWindow<>(inputWindowingStrategy, outputWindowingStrategy));
-  }
-
-  static final class DirectGroupByKeyOnly<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollection<KeyedWorkItem<K, V>>> {
-    @Override
-    public PCollection<KeyedWorkItem<K, V>> expand(PCollection<KV<K, V>> input) {
-      return PCollection.createPrimitiveOutputInternal(
-          input.getPipeline(),
-          WindowingStrategy.globalDefault(),
-          input.isBounded(),
-          KeyedWorkItemCoder.of(
-              GroupByKey.getKeyCoder(input.getCoder()),
-              GroupByKey.getInputValueCoder(input.getCoder()),
-              input.getWindowingStrategy().getWindowFn().windowCoder()));
-    }
-
-    DirectGroupByKeyOnly() {}
-  }
-
-  static final class DirectGroupAlsoByWindow<K, V>
-      extends PTransform<PCollection<KeyedWorkItem<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
-    private final WindowingStrategy<?, ?> inputWindowingStrategy;
-    private final WindowingStrategy<?, ?> outputWindowingStrategy;
-
-    public DirectGroupAlsoByWindow(
-        WindowingStrategy<?, ?> inputWindowingStrategy,
-        WindowingStrategy<?, ?> outputWindowingStrategy) {
-      this.inputWindowingStrategy = inputWindowingStrategy;
-      this.outputWindowingStrategy = outputWindowingStrategy;
-    }
-
-    public WindowingStrategy<?, ?> getInputWindowingStrategy() {
-      return inputWindowingStrategy;
-    }
-
-    private KeyedWorkItemCoder<K, V> getKeyedWorkItemCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
-      // Coder<KV<...>> --> KvCoder<...>
-      checkArgument(
-          inputCoder instanceof KeyedWorkItemCoder,
-          "%s requires a %s<...> but got %s",
-          getClass().getSimpleName(),
-          KvCoder.class.getSimpleName(),
-          inputCoder);
-      @SuppressWarnings("unchecked")
-      KeyedWorkItemCoder<K, V> kvCoder = (KeyedWorkItemCoder<K, V>) inputCoder;
-      return kvCoder;
-    }
-
-    public Coder<V> getValueCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
-      return getKeyedWorkItemCoder(inputCoder).getElementCoder();
-    }
-
-    @Override
-    public PCollection<KV<K, Iterable<V>>> expand(PCollection<KeyedWorkItem<K, V>> input) {
-      KeyedWorkItemCoder<K, V> inputCoder = getKeyedWorkItemCoder(input.getCoder());
-      return PCollection.createPrimitiveOutputInternal(
-          input.getPipeline(),
-          outputWindowingStrategy,
-          input.isBounded(),
-          KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getElementCoder())));
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectMetrics.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectMetrics.java
deleted file mode 100644
index a057260..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectMetrics.java
+++ /dev/null
@@ -1,307 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static java.util.Arrays.asList;
-
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicReference;
-import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
-import org.apache.beam.runners.core.metrics.DistributionData;
-import org.apache.beam.runners.core.metrics.GaugeData;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
-import org.apache.beam.runners.core.metrics.MetricsMap;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.GaugeResult;
-import org.apache.beam.sdk.metrics.MetricFiltering;
-import org.apache.beam.sdk.metrics.MetricKey;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.MetricResult;
-import org.apache.beam.sdk.metrics.MetricResults;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/** Implementation of {@link MetricResults} for the Direct Runner. */
-class DirectMetrics extends MetricResults {
-
-  // TODO: (BEAM-723) Create a shared ExecutorService for maintenance tasks in the DirectRunner.
-  private static final ExecutorService COUNTER_COMMITTER =
-      Executors.newCachedThreadPool(
-          new ThreadFactoryBuilder()
-              .setThreadFactory(MoreExecutors.platformThreadFactory())
-              .setDaemon(true)
-              .setNameFormat("direct-metrics-counter-committer")
-              .build());
-
-  private interface MetricAggregation<UpdateT, ResultT> {
-    UpdateT zero();
-
-    UpdateT combine(Iterable<UpdateT> updates);
-
-    ResultT extract(UpdateT data);
-  }
-
-  /**
-   * Implementation of a metric in the direct runner.
-   *
-   * @param <UpdateT> The type of raw data received and aggregated across updates.
-   * @param <ResultT> The type of result extracted from the data.
-   */
-  private static class DirectMetric<UpdateT, ResultT> {
-    private final MetricAggregation<UpdateT, ResultT> aggregation;
-
-    private final AtomicReference<UpdateT> finishedCommitted;
-
-    private final Object attemptedLock = new Object();
-
-    @GuardedBy("attemptedLock")
-    private volatile UpdateT finishedAttempted;
-
-    private final ConcurrentMap<CommittedBundle<?>, UpdateT> inflightAttempted =
-        new ConcurrentHashMap<>();
-
-    public DirectMetric(MetricAggregation<UpdateT, ResultT> aggregation) {
-      this.aggregation = aggregation;
-      finishedCommitted = new AtomicReference<>(aggregation.zero());
-      finishedAttempted = aggregation.zero();
-    }
-
-    /**
-     * Add the given {@code tentativeCumulative} update to the physical aggregate.
-     *
-     * @param bundle The bundle receiving an update.
-     * @param tentativeCumulative The new cumulative value for the given bundle.
-     */
-    public void updatePhysical(CommittedBundle<?> bundle, UpdateT tentativeCumulative) {
-      // Add (or update) the cumulatiev value for the given bundle.
-      inflightAttempted.put(bundle, tentativeCumulative);
-    }
-
-    /**
-     * Commit a physical value for the given {@code bundle}.
-     *
-     * @param bundle The bundle being committed.
-     * @param finalCumulative The final cumulative value for the given bundle.
-     */
-    @SuppressWarnings("FutureReturnValueIgnored") // direct runner metrics are best-effort;
-    // we choose not to block on async commit
-    public void commitPhysical(final CommittedBundle<?> bundle, final UpdateT finalCumulative) {
-      // To prevent a query from blocking the commit, we perform the commit in two steps.
-      // 1. We perform a non-blocking write to the uncommitted table to make the new value
-      //    available immediately.
-      // 2. We submit a runnable that will commit the update and remove the tentative value in
-      //    a synchronized block.
-      inflightAttempted.put(bundle, finalCumulative);
-      COUNTER_COMMITTER.submit(
-          () -> {
-            synchronized (attemptedLock) {
-              finishedAttempted = aggregation.combine(asList(finishedAttempted, finalCumulative));
-              inflightAttempted.remove(bundle);
-            }
-          });
-    }
-
-    /** Extract the latest values from all attempted and in-progress bundles. */
-    public ResultT extractLatestAttempted() {
-      ArrayList<UpdateT> updates = new ArrayList<>(inflightAttempted.size() + 1);
-      // Within this block we know that will be consistent. Specifically, the only change that can
-      // happen concurrently is the addition of new (larger) values to inflightAttempted.
-      synchronized (attemptedLock) {
-        updates.add(finishedAttempted);
-        updates.addAll(inflightAttempted.values());
-      }
-      return aggregation.extract(aggregation.combine(updates));
-    }
-
-    /**
-     * Commit a logical value for the given {@code bundle}.
-     *
-     * @param bundle The bundle being committed.
-     * @param finalCumulative The final cumulative value for the given bundle.
-     */
-    public void commitLogical(final CommittedBundle<?> bundle, final UpdateT finalCumulative) {
-      UpdateT current;
-      do {
-        current = finishedCommitted.get();
-      } while (!finishedCommitted.compareAndSet(
-          current, aggregation.combine(asList(current, finalCumulative))));
-    }
-
-    /** Extract the value from all successfully committed bundles. */
-    public ResultT extractCommitted() {
-      return aggregation.extract(finishedCommitted.get());
-    }
-  }
-
-  private static final MetricAggregation<Long, Long> COUNTER =
-      new MetricAggregation<Long, Long>() {
-        @Override
-        public Long zero() {
-          return 0L;
-        }
-
-        @Override
-        public Long combine(Iterable<Long> updates) {
-          long value = 0;
-          for (long update : updates) {
-            value += update;
-          }
-          return value;
-        }
-
-        @Override
-        public Long extract(Long data) {
-          return data;
-        }
-      };
-
-  private static final MetricAggregation<DistributionData, DistributionResult> DISTRIBUTION =
-      new MetricAggregation<DistributionData, DistributionResult>() {
-        @Override
-        public DistributionData zero() {
-          return DistributionData.EMPTY;
-        }
-
-        @Override
-        public DistributionData combine(Iterable<DistributionData> updates) {
-          DistributionData result = DistributionData.EMPTY;
-          for (DistributionData update : updates) {
-            result = result.combine(update);
-          }
-          return result;
-        }
-
-        @Override
-        public DistributionResult extract(DistributionData data) {
-          return data.extractResult();
-        }
-      };
-
-  private static final MetricAggregation<GaugeData, GaugeResult> GAUGE =
-      new MetricAggregation<GaugeData, GaugeResult>() {
-        @Override
-        public GaugeData zero() {
-          return GaugeData.empty();
-        }
-
-        @Override
-        public GaugeData combine(Iterable<GaugeData> updates) {
-          GaugeData result = GaugeData.empty();
-          for (GaugeData update : updates) {
-            result = result.combine(update);
-          }
-          return result;
-        }
-
-        @Override
-        public GaugeResult extract(GaugeData data) {
-          return data.extractResult();
-        }
-      };
-
-  /** The current values of counters in memory. */
-  private MetricsMap<MetricKey, DirectMetric<Long, Long>> counters =
-      new MetricsMap<>(unusedKey -> new DirectMetric<>(COUNTER));
-
-  private MetricsMap<MetricKey, DirectMetric<DistributionData, DistributionResult>> distributions =
-      new MetricsMap<>(unusedKey -> new DirectMetric<>(DISTRIBUTION));
-  private MetricsMap<MetricKey, DirectMetric<GaugeData, GaugeResult>> gauges =
-      new MetricsMap<>(unusedKey -> new DirectMetric<>(GAUGE));
-
-  @Override
-  public MetricQueryResults queryMetrics(@Nullable MetricsFilter filter) {
-    ImmutableList.Builder<MetricResult<Long>> counterResults = ImmutableList.builder();
-    for (Entry<MetricKey, DirectMetric<Long, Long>> counter : counters.entries()) {
-      maybeExtractResult(filter, counterResults, counter);
-    }
-    ImmutableList.Builder<MetricResult<DistributionResult>> distributionResults =
-        ImmutableList.builder();
-    for (Entry<MetricKey, DirectMetric<DistributionData, DistributionResult>> distribution :
-        distributions.entries()) {
-      maybeExtractResult(filter, distributionResults, distribution);
-    }
-    ImmutableList.Builder<MetricResult<GaugeResult>> gaugeResults = ImmutableList.builder();
-    for (Entry<MetricKey, DirectMetric<GaugeData, GaugeResult>> gauge : gauges.entries()) {
-      maybeExtractResult(filter, gaugeResults, gauge);
-    }
-
-    return MetricQueryResults.create(
-        counterResults.build(), distributionResults.build(), gaugeResults.build());
-  }
-
-  private <ResultT> void maybeExtractResult(
-      MetricsFilter filter,
-      ImmutableList.Builder<MetricResult<ResultT>> resultsBuilder,
-      Map.Entry<MetricKey, ? extends DirectMetric<?, ResultT>> entry) {
-    if (MetricFiltering.matches(filter, entry.getKey())) {
-      resultsBuilder.add(
-          MetricResult.create(
-              entry.getKey(),
-              entry.getValue().extractCommitted(),
-              entry.getValue().extractLatestAttempted()));
-    }
-  }
-
-  /** Apply metric updates that represent physical counter deltas to the current metric values. */
-  public void updatePhysical(CommittedBundle<?> bundle, MetricUpdates updates) {
-    for (MetricUpdate<Long> counter : updates.counterUpdates()) {
-      counters.get(counter.getKey()).updatePhysical(bundle, counter.getUpdate());
-    }
-    for (MetricUpdate<DistributionData> distribution : updates.distributionUpdates()) {
-      distributions.get(distribution.getKey()).updatePhysical(bundle, distribution.getUpdate());
-    }
-    for (MetricUpdate<GaugeData> gauge : updates.gaugeUpdates()) {
-      gauges.get(gauge.getKey()).updatePhysical(bundle, gauge.getUpdate());
-    }
-  }
-
-  public void commitPhysical(CommittedBundle<?> bundle, MetricUpdates updates) {
-    for (MetricUpdate<Long> counter : updates.counterUpdates()) {
-      counters.get(counter.getKey()).commitPhysical(bundle, counter.getUpdate());
-    }
-    for (MetricUpdate<DistributionData> distribution : updates.distributionUpdates()) {
-      distributions.get(distribution.getKey()).commitPhysical(bundle, distribution.getUpdate());
-    }
-    for (MetricUpdate<GaugeData> gauge : updates.gaugeUpdates()) {
-      gauges.get(gauge.getKey()).commitPhysical(bundle, gauge.getUpdate());
-    }
-  }
-
-  /** Apply metric updates that represent new logical values from a bundle being committed. */
-  public void commitLogical(CommittedBundle<?> bundle, MetricUpdates updates) {
-    for (MetricUpdate<Long> counter : updates.counterUpdates()) {
-      counters.get(counter.getKey()).commitLogical(bundle, counter.getUpdate());
-    }
-    for (MetricUpdate<DistributionData> distribution : updates.distributionUpdates()) {
-      distributions.get(distribution.getKey()).commitLogical(bundle, distribution.getUpdate());
-    }
-    for (MetricUpdate<GaugeData> gauge : updates.gaugeUpdates()) {
-      gauges.get(gauge.getKey()).commitLogical(bundle, gauge.getUpdate());
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectStateAndTimers.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectStateAndTimers.java
deleted file mode 100644
index ca36781..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectStateAndTimers.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.direct.Clock;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.local.StructuralKey;
-
-/**
- * State and Timer access for the {@link ReferenceRunner}.
- *
- * <p>This provides per-key, per-stage access to {@link CopyOnAccessInMemoryStateInternals} and
- * {@link DirectTimerInternals} for transforms that require access to state or timers.
- *
- * <p>This implementation is not thread safe. A new {@link DirectStateAndTimers} must be created for
- * each thread that requires it.
- */
-class DirectStateAndTimers<K> implements StepStateAndTimers<K> {
-  private final StructuralKey<K> key;
-  private final CopyOnAccessInMemoryStateInternals existingState;
-
-  private final Clock clock;
-  private final TransformWatermarks watermarks;
-
-  private CopyOnAccessInMemoryStateInternals<K> stateInternals;
-  private DirectTimerInternals timerInternals;
-
-  DirectStateAndTimers(
-      StructuralKey<K> key,
-      CopyOnAccessInMemoryStateInternals existingState,
-      Clock clock,
-      TransformWatermarks watermarks) {
-    this.key = key;
-    this.existingState = existingState;
-    this.clock = clock;
-    this.watermarks = watermarks;
-  }
-
-  @Override
-  public CopyOnAccessInMemoryStateInternals<K> stateInternals() {
-    if (stateInternals == null) {
-      stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
-    }
-    return stateInternals;
-  }
-
-  @Override
-  public DirectTimerInternals timerInternals() {
-    if (timerInternals == null) {
-      timerInternals = DirectTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
-    }
-    return timerInternals;
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTimerInternals.java
deleted file mode 100644
index 15d7b6c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTimerInternals.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.direct.Clock;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.joda.time.Instant;
-
-/** An implementation of {@link TimerInternals} where all relevant data exists in memory. */
-class DirectTimerInternals implements TimerInternals {
-  private final Clock processingTimeClock;
-  private final TransformWatermarks watermarks;
-  private final TimerUpdateBuilder timerUpdateBuilder;
-
-  public static DirectTimerInternals create(
-      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
-    return new DirectTimerInternals(clock, watermarks, timerUpdateBuilder);
-  }
-
-  private DirectTimerInternals(
-      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
-    this.processingTimeClock = clock;
-    this.watermarks = watermarks;
-    this.timerUpdateBuilder = timerUpdateBuilder;
-  }
-
-  @Override
-  public void setTimer(
-      StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
-    timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, timeDomain));
-  }
-
-  /** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */
-  @Deprecated
-  @Override
-  public void setTimer(TimerData timerData) {
-    timerUpdateBuilder.setTimer(timerData);
-  }
-
-  @Override
-  public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
-    throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
-  }
-
-  /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
-  @Deprecated
-  @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
-    throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
-  }
-
-  /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
-  @Deprecated
-  @Override
-  public void deleteTimer(TimerData timerKey) {
-    timerUpdateBuilder.deletedTimer(timerKey);
-  }
-
-  public TimerUpdate getTimerUpdate() {
-    return timerUpdateBuilder.build();
-  }
-
-  @Override
-  public Instant currentProcessingTime() {
-    return processingTimeClock.now();
-  }
-
-  @Override
-  @Nullable
-  public Instant currentSynchronizedProcessingTime() {
-    return watermarks.getSynchronizedProcessingInputTime();
-  }
-
-  @Override
-  public Instant currentInputWatermarkTime() {
-    return watermarks.getInputWatermark();
-  }
-
-  @Override
-  @Nullable
-  public Instant currentOutputWatermarkTime() {
-    return watermarks.getOutputWatermark();
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTransformExecutor.java
deleted file mode 100644
index b4e742a..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTransformExecutor.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.io.Closeable;
-import java.util.concurrent.Callable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.core.metrics.MetricsContainerImpl;
-import org.apache.beam.sdk.metrics.MetricsEnvironment;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a {@link
- * TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering the result
- * using a registered {@link CompletionCallback}.
- */
-class DirectTransformExecutor<T> implements TransformExecutor {
-  private static final Logger LOG = LoggerFactory.getLogger(DirectTransformExecutor.class);
-
-  static class Factory implements TransformExecutorFactory {
-    private final EvaluationContext context;
-    private final TransformEvaluatorRegistry registry;
-
-    Factory(EvaluationContext context, TransformEvaluatorRegistry registry) {
-      this.context = context;
-      this.registry = registry;
-    }
-
-    @Override
-    public TransformExecutor create(
-        CommittedBundle<?> bundle,
-        PTransformNode transform,
-        CompletionCallback onComplete,
-        TransformExecutorService executorService) {
-      return new DirectTransformExecutor<>(
-          context, registry, bundle, transform, onComplete, executorService);
-    }
-  }
-
-  private final TransformEvaluatorRegistry evaluatorRegistry;
-
-  /** The transform that will be evaluated. */
-  private final PTransformNode transform;
-  /** The inputs this {@link DirectTransformExecutor} will deliver to the transform. */
-  private final CommittedBundle<T> inputBundle;
-
-  private final CompletionCallback onComplete;
-  private final TransformExecutorService transformEvaluationState;
-  private final EvaluationContext context;
-
-  @VisibleForTesting
-  DirectTransformExecutor(
-      EvaluationContext context,
-      TransformEvaluatorRegistry factory,
-      CommittedBundle<T> inputBundle,
-      PTransformNode transform,
-      CompletionCallback completionCallback,
-      TransformExecutorService transformEvaluationState) {
-    this.evaluatorRegistry = factory;
-
-    this.inputBundle = inputBundle;
-    this.transform = transform;
-
-    this.onComplete = completionCallback;
-
-    this.transformEvaluationState = transformEvaluationState;
-    this.context = context;
-  }
-
-  @Override
-  public void run() {
-    MetricsContainerImpl metricsContainer = new MetricsContainerImpl(transform.getId());
-    try (Closeable metricsScope = MetricsEnvironment.scopedMetricsContainer(metricsContainer)) {
-      TransformEvaluator<T> evaluator = evaluatorRegistry.forApplication(transform, inputBundle);
-      if (evaluator == null) {
-        onComplete.handleEmpty(transform);
-        // Nothing to do
-        return;
-      }
-
-      processElements(evaluator, metricsContainer);
-
-      finishBundle(evaluator, metricsContainer);
-    } catch (Exception e) {
-      onComplete.handleException(inputBundle, e);
-      if (e instanceof RuntimeException) {
-        throw (RuntimeException) e;
-      }
-      throw new RuntimeException(e);
-    } catch (Error err) {
-      LOG.error("Error occurred within {}", this, err);
-      onComplete.handleError(err);
-      throw err;
-    } finally {
-      // Report the physical metrics from the end of this step.
-      context.getMetrics().commitPhysical(inputBundle, metricsContainer.getCumulative());
-
-      transformEvaluationState.complete(this);
-    }
-  }
-
-  /** Processes all the elements in the input bundle using the transform evaluator. */
-  private void processElements(
-      TransformEvaluator<T> evaluator, MetricsContainerImpl metricsContainer) throws Exception {
-    if (inputBundle != null) {
-      for (WindowedValue<T> value : inputBundle.getElements()) {
-        evaluator.processElement(value);
-
-        // Report the physical metrics after each element
-        MetricUpdates deltas = metricsContainer.getUpdates();
-        if (deltas != null) {
-          context.getMetrics().updatePhysical(inputBundle, deltas);
-          metricsContainer.commitUpdates();
-        }
-      }
-    }
-  }
-
-  /**
-   * Finishes processing the input bundle and commit the result using the {@link
-   * CompletionCallback}.
-   *
-   * @return the {@link TransformResult} produced by {@link TransformEvaluator#finishBundle()}
-   */
-  private TransformResult<T> finishBundle(
-      TransformEvaluator<T> evaluator, MetricsContainerImpl metricsContainer) throws Exception {
-    TransformResult<T> result =
-        evaluator.finishBundle().withLogicalMetricUpdates(metricsContainer.getCumulative());
-    onComplete.handleResult(inputBundle, result);
-    return result;
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DisplayDataValidator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DisplayDataValidator.java
deleted file mode 100644
index c01dd62..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DisplayDataValidator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.display.HasDisplayData;
-
-/**
- * Validate correct implementation of {@link DisplayData} by evaluating {@link
- * HasDisplayData#populateDisplayData(DisplayData.Builder)} during pipeline construction.
- */
-class DisplayDataValidator {
-  // Do not instantiate
-  private DisplayDataValidator() {}
-
-  static void validatePipeline(Pipeline pipeline) {
-    validateTransforms(pipeline);
-  }
-
-  static void validateOptions(PipelineOptions options) {
-    evaluateDisplayData(options);
-  }
-
-  private static void validateTransforms(Pipeline pipeline) {
-    pipeline.traverseTopologically(Visitor.INSTANCE);
-  }
-
-  private static void evaluateDisplayData(HasDisplayData component) {
-    DisplayData.from(component);
-  }
-
-  private static class Visitor extends Pipeline.PipelineVisitor.Defaults {
-    private static final Visitor INSTANCE = new Visitor();
-
-    @Override
-    public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-      if (!node.isRootNode()) {
-        evaluateDisplayData(node.getTransform());
-      }
-
-      return CompositeBehavior.ENTER_TRANSFORM;
-    }
-
-    @Override
-    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-      evaluateDisplayData(node.getTransform());
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EmptyInputProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EmptyInputProvider.java
deleted file mode 100644
index 82edcef..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EmptyInputProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-
-/** A {@link RootInputProvider} that provides no input bundles. */
-class EmptyInputProvider implements RootInputProvider<Void> {
-  EmptyInputProvider() {}
-
-  /**
-   * {@inheritDoc}.
-   *
-   * <p>Returns an empty collection.
-   */
-  @Override
-  public Collection<CommittedBundle<Void>> getInitialInputs(
-      PTransformNode transform, int targetParallelism) {
-    return Collections.emptyList();
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContext.java
deleted file mode 100644
index a1b57a9..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContext.java
+++ /dev/null
@@ -1,333 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.Clock;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.WatermarkManager;
-import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.joda.time.Instant;
-
-/**
- * The evaluation context for a specific pipeline being executed by the {@code DirectRunner}.
- * Contains state shared within the execution across all transforms.
- *
- * <p>{@link EvaluationContext} contains shared state for an execution of the {@code DirectRunner}
- * that can be used while evaluating a {@link PTransform}. This consists of views into underlying
- * state and watermark implementations, access to read and write {@link PCollectionView
- * PCollectionViews}, and managing the {@link DirectStateAndTimers ExecutionContexts}. This includes
- * executing callbacks asynchronously when state changes to the appropriate point (e.g. when a
- * {@link PCollectionView} is requested and known to be empty).
- *
- * <p>{@link EvaluationContext} also handles results by committing finalizing bundles based on the
- * current global state and updating the global state appropriately. This includes updating the
- * per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that can be
- * executed.
- */
-class EvaluationContext {
-  /** The graph representing this {@link Pipeline}. */
-  private final ExecutableGraph<PTransformNode, ? super PCollectionNode> graph;
-
-  private final Clock clock;
-
-  private final BundleFactory bundleFactory;
-
-  /** The current processing time and event time watermarks and timers. */
-  private final WatermarkManager<PTransformNode, ? super PCollectionNode> watermarkManager;
-
-  /** Executes callbacks based on the progression of the watermark. */
-  private final WatermarkCallbackExecutor callbackExecutor;
-
-  /** The stateInternals of the world, by applied PTransform and key. */
-  private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals>
-      applicationStateInternals;
-
-  private final DirectMetrics metrics;
-
-  private final Set<PCollectionNode> keyedPValues;
-
-  public static EvaluationContext create(
-      Clock clock,
-      BundleFactory bundleFactory,
-      ExecutableGraph<PTransformNode, ? super PCollectionNode> graph,
-      Set<PCollectionNode> keyedPValues) {
-    return new EvaluationContext(clock, bundleFactory, graph, keyedPValues);
-  }
-
-  private EvaluationContext(
-      Clock clock,
-      BundleFactory bundleFactory,
-      ExecutableGraph<PTransformNode, ? super PCollectionNode> graph,
-      Set<PCollectionNode> keyedPValues) {
-    this.clock = clock;
-    this.bundleFactory = checkNotNull(bundleFactory);
-    this.graph = checkNotNull(graph);
-    this.keyedPValues = keyedPValues;
-
-    this.watermarkManager = WatermarkManager.create(clock, graph, PTransformNode::getId);
-
-    this.applicationStateInternals = new ConcurrentHashMap<>();
-    this.metrics = new DirectMetrics();
-
-    this.callbackExecutor = WatermarkCallbackExecutor.create(MoreExecutors.directExecutor());
-  }
-
-  public void initialize(
-      Map<PTransformNode, ? extends Iterable<CommittedBundle<?>>> initialInputs) {
-    watermarkManager.initialize((Map) initialInputs);
-  }
-
-  /**
-   * Handle the provided {@link TransformResult}, produced after evaluating the provided {@link
-   * CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
-   *
-   * <p>The result is the output of running the transform contained in the {@link TransformResult}
-   * on the contents of the provided bundle.
-   *
-   * @param completedBundle the bundle that was processed to produce the result. Potentially {@code
-   *     null} if the transform that produced the result is a root transform
-   * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
-   *     or an empty iterable if no timers were delivered
-   * @param result the result of evaluating the input bundle
-   * @return the committed bundles contained within the handled {@code result}
-   */
-  public CommittedResult<PTransformNode> handleResult(
-      CommittedBundle<?> completedBundle,
-      Iterable<TimerData> completedTimers,
-      TransformResult<?> result) {
-    Iterable<? extends CommittedBundle<?>> committedBundles =
-        commitBundles(result.getOutputBundles());
-    metrics.commitLogical(completedBundle, result.getLogicalMetricUpdates());
-
-    // Update watermarks and timers
-    EnumSet<OutputType> outputTypes = EnumSet.copyOf(result.getOutputTypes());
-    if (Iterables.isEmpty(committedBundles)) {
-      outputTypes.remove(OutputType.BUNDLE);
-    } else {
-      outputTypes.add(OutputType.BUNDLE);
-    }
-    CommittedResult<PTransformNode> committedResult =
-        CommittedResult.create(
-            result, getUnprocessedInput(completedBundle, result), committedBundles, outputTypes);
-    // Update state internals
-    CopyOnAccessInMemoryStateInternals theirState = result.getState();
-    if (theirState != null) {
-      CopyOnAccessInMemoryStateInternals committedState = theirState.commit();
-      StepAndKey stepAndKey = StepAndKey.of(result.getTransform(), completedBundle.getKey());
-      if (!committedState.isEmpty()) {
-        applicationStateInternals.put(stepAndKey, committedState);
-      } else {
-        applicationStateInternals.remove(stepAndKey);
-      }
-    }
-    // Watermarks are updated last to ensure visibility of any global state before progress is
-    // permitted
-    watermarkManager.updateWatermarks(
-        completedBundle,
-        result.getTimerUpdate().withCompletedTimers(completedTimers),
-        committedResult.getExecutable(),
-        committedResult.getUnprocessedInputs().orNull(),
-        committedResult.getOutputs(),
-        result.getWatermarkHold());
-    return committedResult;
-  }
-
-  /**
-   * Returns an {@link Optional} containing a bundle which contains all of the unprocessed elements
-   * that were not processed from the {@code completedBundle}. If all of the elements of the {@code
-   * completedBundle} were processed, or if {@code completedBundle} is null, returns an absent
-   * {@link Optional}.
-   */
-  private Optional<? extends CommittedBundle<?>> getUnprocessedInput(
-      CommittedBundle<?> completedBundle, TransformResult<?> result) {
-    if (completedBundle == null || Iterables.isEmpty(result.getUnprocessedElements())) {
-      return Optional.absent();
-    }
-    CommittedBundle<?> residual =
-        completedBundle.withElements((Iterable) result.getUnprocessedElements());
-    return Optional.of(residual);
-  }
-
-  private Iterable<? extends CommittedBundle<?>> commitBundles(
-      Iterable<? extends UncommittedBundle<?>> bundles) {
-    ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
-    for (UncommittedBundle<?> inProgress : bundles) {
-      PTransformNode producing = graph.getProducer(inProgress.getPCollection());
-      TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
-      CommittedBundle<?> committed =
-          inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
-      // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
-      // filter them out
-      if (!Iterables.isEmpty(committed.getElements())) {
-        completed.add(committed);
-      }
-    }
-    return completed.build();
-  }
-
-  private void fireAllAvailableCallbacks() {
-    for (PTransformNode transform : graph.getExecutables()) {
-      fireAvailableCallbacks(transform);
-    }
-  }
-
-  private void fireAvailableCallbacks(PTransformNode producingTransform) {
-    TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
-    Instant outputWatermark = watermarks.getOutputWatermark();
-    callbackExecutor.fireForWatermark(producingTransform, outputWatermark);
-  }
-
-  /** Create a {@link UncommittedBundle} for use by a source. */
-  public <T> UncommittedBundle<T> createRootBundle() {
-    return bundleFactory.createRootBundle();
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} whose elements belong to the specified {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createBundle(PCollectionNode output) {
-    return bundleFactory.createBundle(output);
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
-   * {@code DirectGroupByKeyOnly} {@link PTransform PTransforms}.
-   */
-  public <K, T> UncommittedBundle<T> createKeyedBundle(
-      StructuralKey<K> key, PCollectionNode output) {
-    return bundleFactory.createKeyedBundle(key, output);
-  }
-
-  /** Indicate whether or not this {@link PCollection} has been determined to be keyed. */
-  public <T> boolean isKeyed(PCollectionNode pValue) {
-    return keyedPValues.contains(pValue);
-  }
-
-  /**
-   * Schedule a callback to be executed after output would be produced for the given window if there
-   * had been input.
-   *
-   * <p>Output would be produced when the watermark for a {@link PValue} passes the point at which
-   * the trigger for the specified window (with the specified windowing strategy) must have fired
-   * from the perspective of that {@link PValue}, as specified by the value of {@link
-   * Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the {@link
-   * WindowingStrategy}. When the callback has fired, either values will have been produced for a
-   * key in that window, the window is empty, or all elements in the window are late. The callback
-   * will be executed regardless of whether values have been produced.
-   */
-  public void scheduleAfterOutputWouldBeProduced(
-      PCollectionNode value,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    PTransformNode producing = graph.getProducer(value);
-    callbackExecutor.callOnWindowExpiration(producing, window, windowingStrategy, runnable);
-
-    fireAvailableCallbacks(producing);
-  }
-
-  /** Get a {@link DirectStateAndTimers} for the provided {@link PTransformNode} and key. */
-  public <K> StepStateAndTimers<K> getStateAndTimers(
-      PTransformNode application, StructuralKey<K> key) {
-    StepAndKey stepAndKey = StepAndKey.of(application, key);
-    return new DirectStateAndTimers<>(
-        key,
-        applicationStateInternals.get(stepAndKey),
-        clock,
-        watermarkManager.getWatermarks(application));
-  }
-
-  /** Returns all of the steps in this {@link Pipeline}. */
-  Collection<PTransformNode> getSteps() {
-    return graph.getExecutables();
-  }
-
-  /** Returns the metrics container for this pipeline. */
-  public DirectMetrics getMetrics() {
-    return metrics;
-  }
-
-  @VisibleForTesting
-  void forceRefresh() {
-    watermarkManager.refreshAll();
-    fireAllAvailableCallbacks();
-  }
-
-  /**
-   * Extracts all timers that have been fired and have not already been extracted.
-   *
-   * <p>This is a destructive operation. Timers will only appear in the result of this method once
-   * for each time they are set.
-   */
-  public Collection<FiredTimers<PTransformNode>> extractFiredTimers() {
-    forceRefresh();
-    return watermarkManager.extractFiredTimers();
-  }
-
-  /** Returns true if the step will not produce additional output. */
-  public boolean isDone(PTransformNode transform) {
-    // the PTransform is done only if watermark is at the max value
-    Instant stepWatermark = watermarkManager.getWatermarks(transform).getOutputWatermark();
-    return !stepWatermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE);
-  }
-
-  /** Returns true if all steps are done. */
-  public boolean isDone() {
-    for (PTransformNode transform : graph.getExecutables()) {
-      if (!isDone(transform)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  public Instant now() {
-    return clock.now();
-  }
-
-  Clock getClock() {
-    return clock;
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContextStepStateAndTimersProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContextStepStateAndTimersProvider.java
deleted file mode 100644
index f6ef49c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContextStepStateAndTimersProvider.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.local.StructuralKey;
-
-/** A {@link StepStateAndTimers.Provider} that uses an {@link EvaluationContext}. */
-class EvaluationContextStepStateAndTimersProvider implements StepStateAndTimers.Provider {
-  public static StepStateAndTimers.Provider forContext(EvaluationContext context) {
-    return new EvaluationContextStepStateAndTimersProvider(context);
-  }
-
-  private final EvaluationContext context;
-
-  private EvaluationContextStepStateAndTimersProvider(EvaluationContext context) {
-    this.context = context;
-  }
-
-  @Override
-  public <K> StepStateAndTimers<K> forStepAndKey(PTransformNode transform, StructuralKey<K> key) {
-    return context.getStateAndTimers(transform, key);
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceFactory.java
deleted file mode 100644
index 8b097ad..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.concurrent.ExecutorService;
-
-/**
- * A factory that creates {@link ExecutorService ExecutorServices}. {@link ExecutorService
- * ExecutorServices} created by this factory should be independent of one another (e.g., if any
- * executor is shut down the remaining executors should continue to process work).
- */
-interface ExecutorServiceFactory {
-  /** Create a new {@link ExecutorService}. */
-  ExecutorService create();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceParallelExecutor.java
deleted file mode 100644
index 1a7a99a..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceParallelExecutor.java
+++ /dev/null
@@ -1,392 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.local.ExecutionDriver;
-import org.apache.beam.runners.local.ExecutionDriver.DriverState;
-import org.apache.beam.runners.local.PipelineMessageReceiver;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.CacheBuilder;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.CacheLoader;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.LoadingCache;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.RemovalListener;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An {@link PipelineExecutor} that uses an underlying {@link ExecutorService} and {@link
- * EvaluationContext} to execute a {@link Pipeline}.
- */
-final class ExecutorServiceParallelExecutor
-    implements PipelineExecutor,
-        BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode> {
-  private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
-
-  private final int targetParallelism;
-  private final ExecutorService executorService;
-
-  private final RootProviderRegistry rootRegistry;
-  private final TransformEvaluatorRegistry transformRegistry;
-
-  private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-  private final EvaluationContext evaluationContext;
-
-  private final TransformExecutorFactory executorFactory;
-  private final TransformExecutorService parallelExecutorService;
-  private final LoadingCache<StepAndKey, TransformExecutorService> serialExecutorServices;
-
-  private final QueueMessageReceiver visibleUpdates;
-
-  private AtomicReference<State> pipelineState = new AtomicReference<>(State.RUNNING);
-
-  public static ExecutorServiceParallelExecutor create(
-      int targetParallelism,
-      RootProviderRegistry rootRegistry,
-      TransformEvaluatorRegistry transformRegistry,
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      EvaluationContext context) {
-    return new ExecutorServiceParallelExecutor(
-        targetParallelism, rootRegistry, transformRegistry, graph, context);
-  }
-
-  private ExecutorServiceParallelExecutor(
-      int targetParallelism,
-      RootProviderRegistry rootRegistry,
-      TransformEvaluatorRegistry transformRegistry,
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      EvaluationContext context) {
-    this.targetParallelism = targetParallelism;
-    // Don't use Daemon threads for workers. The Pipeline should continue to execute even if there
-    // are no other active threads (for example, because waitUntilFinish was not called)
-    this.executorService =
-        Executors.newFixedThreadPool(
-            targetParallelism,
-            new ThreadFactoryBuilder()
-                .setThreadFactory(MoreExecutors.platformThreadFactory())
-                .setNameFormat("direct-runner-worker")
-                .build());
-    this.rootRegistry = rootRegistry;
-    this.transformRegistry = transformRegistry;
-    this.graph = graph;
-    this.evaluationContext = context;
-
-    // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
-    // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
-    // which stops the TransformExecutorServices from being prematurely garbage collected
-    serialExecutorServices =
-        CacheBuilder.newBuilder()
-            .weakValues()
-            .removalListener(shutdownExecutorServiceListener())
-            .build(serialTransformExecutorServiceCacheLoader());
-
-    this.visibleUpdates = new QueueMessageReceiver();
-
-    parallelExecutorService = TransformExecutorServices.parallel(executorService);
-    executorFactory = new DirectTransformExecutor.Factory(context, transformRegistry);
-  }
-
-  private CacheLoader<StepAndKey, TransformExecutorService>
-      serialTransformExecutorServiceCacheLoader() {
-    return new CacheLoader<StepAndKey, TransformExecutorService>() {
-      @Override
-      public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
-        return TransformExecutorServices.serial(executorService);
-      }
-    };
-  }
-
-  private RemovalListener<StepAndKey, TransformExecutorService> shutdownExecutorServiceListener() {
-    return notification -> {
-      TransformExecutorService service = notification.getValue();
-      if (service != null) {
-        service.shutdown();
-      }
-    };
-  }
-
-  @Override
-  // TODO: [BEAM-4563] Pass Future back to consumer to check for async errors
-  @SuppressWarnings("FutureReturnValueIgnored")
-  public void start() {
-    int numTargetSplits = Math.max(3, targetParallelism);
-    ImmutableMap.Builder<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>>
-        pendingRootBundles = ImmutableMap.builder();
-    for (PTransformNode root : graph.getRootTransforms()) {
-      ConcurrentLinkedQueue<CommittedBundle<?>> pending = new ConcurrentLinkedQueue<>();
-      try {
-        Collection<CommittedBundle<?>> initialInputs =
-            rootRegistry.getInitialInputs(root, numTargetSplits);
-        pending.addAll(initialInputs);
-      } catch (Exception e) {
-        throw UserCodeException.wrap(e);
-      }
-      pendingRootBundles.put(root, pending);
-    }
-    evaluationContext.initialize(pendingRootBundles.build());
-    final ExecutionDriver executionDriver =
-        QuiescenceDriver.create(
-            evaluationContext, graph, this, visibleUpdates, pendingRootBundles.build());
-    executorService.submit(
-        new Runnable() {
-          @Override
-          public void run() {
-            DriverState drive = executionDriver.drive();
-            if (drive.isTermainal()) {
-              State newPipelineState = State.UNKNOWN;
-              switch (drive) {
-                case FAILED:
-                  newPipelineState = State.FAILED;
-                  break;
-                case SHUTDOWN:
-                  newPipelineState = State.DONE;
-                  break;
-                case CONTINUE:
-                  throw new IllegalStateException(
-                      String.format("%s should not be a terminal state", DriverState.CONTINUE));
-                default:
-                  throw new IllegalArgumentException(
-                      String.format("Unknown %s %s", DriverState.class.getSimpleName(), drive));
-              }
-              shutdownIfNecessary(newPipelineState);
-            } else {
-              executorService.submit(this);
-            }
-          }
-        });
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void process(
-      CommittedBundle<?> bundle, PTransformNode consumer, CompletionCallback onComplete) {
-    evaluateBundle(consumer, bundle, onComplete);
-  }
-
-  private <T> void evaluateBundle(
-      final PTransformNode 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 DirectTransformExecutor callable. Follow-up TransformExecutors
-      // (scheduled due to the completion of another DirectTransformExecutor) are provided to the
-      // ExecutorService before the Earlier DirectTransformExecutor callable completes.
-      transformExecutor = serialExecutorServices.getUnchecked(stepAndKey);
-    } else {
-      transformExecutor = parallelExecutorService;
-    }
-
-    TransformExecutor callable =
-        executorFactory.create(bundle, transform, onComplete, transformExecutor);
-    if (!pipelineState.get().isTerminal()) {
-      transformExecutor.schedule(callable);
-    }
-  }
-
-  private boolean isKeyed(PCollectionNode pvalue) {
-    return evaluationContext.isKeyed(pvalue);
-  }
-
-  @Override
-  public State waitUntilFinish(Duration duration) throws Exception {
-    Instant completionTime;
-    if (duration.equals(Duration.ZERO)) {
-      completionTime = new Instant(Long.MAX_VALUE);
-    } else {
-      completionTime = Instant.now().plus(duration);
-    }
-
-    VisibleExecutorUpdate update = null;
-    while (Instant.now().isBefore(completionTime)
-        && (update == null || isTerminalStateUpdate(update))) {
-      // Get an update; don't block forever if another thread has handled it. The call to poll will
-      // wait the entire timeout; this call primarily exists to relinquish any core.
-      update = visibleUpdates.tryNext(Duration.millis(25L));
-      if (update == null && pipelineState.get().isTerminal()) {
-        // there are no updates to process and no updates will ever be published because the
-        // executor is shutdown
-        return pipelineState.get();
-      } else if (update != null && update.thrown.isPresent()) {
-        Throwable thrown = update.thrown.get();
-        if (thrown instanceof Exception) {
-          throw (Exception) thrown;
-        } else if (thrown instanceof Error) {
-          throw (Error) thrown;
-        } else {
-          throw new Exception("Unknown Type of Throwable", thrown);
-        }
-      }
-    }
-    return pipelineState.get();
-  }
-
-  @Override
-  public State getPipelineState() {
-    return pipelineState.get();
-  }
-
-  private boolean isTerminalStateUpdate(VisibleExecutorUpdate update) {
-    return !(update.getNewState() == null && update.getNewState().isTerminal());
-  }
-
-  @Override
-  public void stop() {
-    shutdownIfNecessary(State.CANCELLED);
-    visibleUpdates.cancelled();
-  }
-
-  private void shutdownIfNecessary(State newState) {
-    if (!newState.isTerminal()) {
-      return;
-    }
-    LOG.debug("Pipeline has terminated. Shutting down.");
-
-    final Collection<Exception> errors = new ArrayList<>();
-    // Stop accepting new work before shutting down the executor. This ensures that thread don't try
-    // to add work to the shutdown executor.
-    try {
-      serialExecutorServices.invalidateAll();
-    } catch (final RuntimeException re) {
-      errors.add(re);
-    }
-    try {
-      serialExecutorServices.cleanUp();
-    } catch (final RuntimeException re) {
-      errors.add(re);
-    }
-    try {
-      parallelExecutorService.shutdown();
-    } catch (final RuntimeException re) {
-      errors.add(re);
-    }
-    try {
-      executorService.shutdown();
-    } catch (final RuntimeException re) {
-      errors.add(re);
-    }
-    try {
-      transformRegistry.cleanup();
-    } catch (final Exception e) {
-      errors.add(e);
-    }
-    pipelineState.compareAndSet(State.RUNNING, newState); // ensure we hit a terminal node
-    if (!errors.isEmpty()) {
-      final IllegalStateException exception =
-          new IllegalStateException(
-              "Error"
-                  + (errors.size() == 1 ? "" : "s")
-                  + " during executor shutdown:\n"
-                  + errors.stream()
-                      .map(Exception::getMessage)
-                      .collect(Collectors.joining("\n- ", "- ", "")));
-      visibleUpdates.failed(exception);
-      throw exception;
-    }
-  }
-
-  /**
-   * An update of interest to the user. Used in {@link #waitUntilFinish} to decide whether to return
-   * normally or throw an exception.
-   */
-  private static class VisibleExecutorUpdate {
-    private final Optional<? extends Throwable> thrown;
-    @Nullable private final State newState;
-
-    public static VisibleExecutorUpdate fromException(Exception e) {
-      return new VisibleExecutorUpdate(null, e);
-    }
-
-    public static VisibleExecutorUpdate fromError(Error err) {
-      return new VisibleExecutorUpdate(State.FAILED, err);
-    }
-
-    public static VisibleExecutorUpdate finished() {
-      return new VisibleExecutorUpdate(State.DONE, null);
-    }
-
-    public static VisibleExecutorUpdate cancelled() {
-      return new VisibleExecutorUpdate(State.CANCELLED, null);
-    }
-
-    private VisibleExecutorUpdate(State newState, @Nullable Throwable exception) {
-      this.thrown = Optional.fromNullable(exception);
-      this.newState = newState;
-    }
-
-    State getNewState() {
-      return newState;
-    }
-  }
-
-  private static class QueueMessageReceiver implements PipelineMessageReceiver {
-    // If the type of BlockingQueue changes, ensure the findbugs filter is updated appropriately
-    private final BlockingQueue<VisibleExecutorUpdate> updates = new LinkedBlockingQueue<>();
-
-    @Override
-    public void failed(Exception e) {
-      updates.offer(VisibleExecutorUpdate.fromException(e));
-    }
-
-    @Override
-    public void failed(Error e) {
-      updates.offer(VisibleExecutorUpdate.fromError(e));
-    }
-
-    @Override
-    public void cancelled() {
-      updates.offer(VisibleExecutorUpdate.cancelled());
-    }
-
-    @Override
-    public void completed() {
-      updates.offer(VisibleExecutorUpdate.finished());
-    }
-
-    /** Try to get the next unconsumed message in this {@link QueueMessageReceiver}. */
-    @Nullable
-    private VisibleExecutorUpdate tryNext(Duration timeout) throws InterruptedException {
-      return updates.poll(timeout.getMillis(), TimeUnit.MILLISECONDS);
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactory.java
deleted file mode 100644
index 31654e0..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactory.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link Flatten} {@link
- * PTransform}.
- */
-class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
-  private final BundleFactory bundleFactory;
-  private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
-  FlattenEvaluatorFactory(
-      ExecutableGraph<PTransformNode, PCollectionNode> graph, BundleFactory bundleFactory) {
-    this.bundleFactory = bundleFactory;
-    this.graph = graph;
-  }
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator = createInMemoryEvaluator(application);
-    return evaluator;
-  }
-
-  @Override
-  public void cleanup() {}
-
-  private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
-      final PTransformNode transform) {
-    return new FlattenEvaluator<>(transform);
-  }
-
-  private class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
-    private final PTransformNode transform;
-    private final UncommittedBundle<InputT> bundle;
-
-    FlattenEvaluator(PTransformNode transform) {
-      this.transform = transform;
-      PCollectionNode output = getOnlyElement(graph.getProduced(transform));
-      bundle = bundleFactory.createBundle(output);
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> element) {
-      bundle.add(element);
-    }
-
-    @Override
-    public TransformResult<InputT> finishBundle() {
-      return StepTransformResult.<InputT>withoutHold(transform).addOutput(bundle).build();
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupAlsoByWindowEvaluatorFactory.java
deleted file mode 100644
index 1c73908..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupAlsoByWindowEvaluatorFactory.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.runners.core.GroupAlsoByWindowsAggregators;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.OutputWindowedValue;
-import org.apache.beam.runners.core.ReduceFnRunner;
-import org.apache.beam.runners.core.SystemReduceFn;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.construction.RehydratedComponents;
-import org.apache.beam.runners.core.construction.TriggerTranslation;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
-import org.apache.beam.runners.core.triggers.TriggerStateMachines;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.portable.DirectGroupByKey.DirectGroupAlsoByWindow;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableLikeCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowTracing;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.joda.time.Instant;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@code
- * DirectGroupAlsoByWindow} {@link PTransform}.
- */
-class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
-  private final BundleFactory bundleFactory;
-  private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-  private final Components components;
-  private final StepStateAndTimers.Provider stp;
-
-  GroupAlsoByWindowEvaluatorFactory(
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      Components components,
-      BundleFactory bundleFactory,
-      StepStateAndTimers.Provider stp) {
-    this.bundleFactory = bundleFactory;
-    this.graph = graph;
-    this.components = components;
-    this.stp = stp;
-  }
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) {
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator =
-        createEvaluator(application, (CommittedBundle) inputBundle);
-    return evaluator;
-  }
-
-  @Override
-  public void cleanup() {}
-
-  private <K, V> TransformEvaluator<KeyedWorkItem<K, V>> createEvaluator(
-      PTransformNode application, CommittedBundle<KeyedWorkItem<K, V>> inputBundle) {
-    @SuppressWarnings("unchecked")
-    StructuralKey<K> key = (StructuralKey<K>) inputBundle.getKey();
-    return new GroupAlsoByWindowEvaluator<>(
-        bundleFactory, key, application, graph, components, stp.forStepAndKey(application, key));
-  }
-
-  /**
-   * A transform evaluator for the pseudo-primitive {@code DirectGroupAlsoByWindow}. The window of
-   * the input {@link KeyedWorkItem} is ignored; it should be in the global window, as element
-   * windows are reified in the {@link KeyedWorkItem#elementsIterable()}.
-   *
-   * @see GroupByKeyViaGroupByKeyOnly
-   */
-  private static class GroupAlsoByWindowEvaluator<K, V>
-      implements TransformEvaluator<KeyedWorkItem<K, V>> {
-    private final BundleFactory bundleFactory;
-
-    private final PTransformNode application;
-    private final PCollectionNode outputCollection;
-
-    private final StructuralKey<?> key;
-
-    private final CopyOnAccessInMemoryStateInternals<K> stateInternals;
-    private final DirectTimerInternals timerInternals;
-    private final WindowingStrategy<?, BoundedWindow> windowingStrategy;
-
-    private final Collection<UncommittedBundle<?>> outputBundles;
-
-    private final SystemReduceFn<K, V, Iterable<V>, Iterable<V>, BoundedWindow> reduceFn;
-    private final Counter droppedDueToLateness;
-
-    private GroupAlsoByWindowEvaluator(
-        BundleFactory bundleFactory,
-        StructuralKey<K> key,
-        PTransformNode application,
-        ExecutableGraph<PTransformNode, PCollectionNode> graph,
-        Components components,
-        StepStateAndTimers<K> stp) {
-      this.bundleFactory = bundleFactory;
-      this.application = application;
-      this.outputCollection = getOnlyElement(graph.getProduced(application));
-      this.key = key;
-
-      this.stateInternals = stp.stateInternals();
-      this.timerInternals = stp.timerInternals();
-
-      PCollectionNode inputCollection = getOnlyElement(graph.getPerElementInputs(application));
-      try {
-        windowingStrategy =
-            (WindowingStrategy<?, BoundedWindow>)
-                RehydratedComponents.forComponents(components)
-                    .getWindowingStrategy(
-                        inputCollection.getPCollection().getWindowingStrategyId());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      outputBundles = new ArrayList<>();
-
-      Coder<V> valueCoder;
-      try {
-        Coder<WindowedValue<KV<K, Iterable<V>>>> windowedValueCoder =
-            WireCoders.instantiateRunnerWireCoder(outputCollection, components);
-        checkArgument(windowedValueCoder instanceof WindowedValue.WindowedValueCoder);
-        Coder<KV<K, Iterable<V>>> outputKvCoder =
-            ((WindowedValueCoder<KV<K, Iterable<V>>>) windowedValueCoder).getValueCoder();
-        checkArgument(outputKvCoder instanceof KvCoder);
-        Coder<Iterable<V>> iterVCoder = ((KvCoder<K, Iterable<V>>) outputKvCoder).getValueCoder();
-        checkArgument(iterVCoder instanceof IterableLikeCoder);
-        valueCoder = ((IterableLikeCoder<V, ?>) iterVCoder).getElemCoder();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-
-      reduceFn = SystemReduceFn.buffering(valueCoder);
-      droppedDueToLateness =
-          Metrics.counter(
-              GroupAlsoByWindowEvaluator.class,
-              GroupAlsoByWindowsAggregators.DROPPED_DUE_TO_LATENESS_COUNTER);
-    }
-
-    @Override
-    public void processElement(WindowedValue<KeyedWorkItem<K, V>> element) throws Exception {
-      KeyedWorkItem<K, V> workItem = element.getValue();
-
-      UncommittedBundle<KV<K, Iterable<V>>> bundle =
-          bundleFactory.createKeyedBundle(this.key, outputCollection);
-      outputBundles.add(bundle);
-      RunnerApi.Trigger runnerApiTrigger =
-          TriggerTranslation.toProto(windowingStrategy.getTrigger());
-      ReduceFnRunner<K, V, Iterable<V>, BoundedWindow> reduceFnRunner =
-          new ReduceFnRunner<>(
-              workItem.key(),
-              windowingStrategy,
-              ExecutableTriggerStateMachine.create(
-                  TriggerStateMachines.stateMachineForTrigger(runnerApiTrigger)),
-              stateInternals,
-              timerInternals,
-              new OutputWindowedValueToBundle<>(bundle),
-              null,
-              reduceFn,
-              null);
-
-      // Drop any elements within expired windows
-      reduceFnRunner.processElements(
-          dropExpiredWindows(workItem.key(), workItem.elementsIterable(), timerInternals));
-      reduceFnRunner.onTimers(workItem.timersIterable());
-      reduceFnRunner.persist();
-    }
-
-    @Override
-    public TransformResult<KeyedWorkItem<K, V>> finishBundle() throws Exception {
-      // State is initialized within the constructor. It can never be null.
-      CopyOnAccessInMemoryStateInternals<?> state = stateInternals.commit();
-      return StepTransformResult.<KeyedWorkItem<K, V>>withHold(
-              application, state.getEarliestWatermarkHold())
-          .withState(state)
-          .addOutput(outputBundles)
-          .withTimerUpdate(timerInternals.getTimerUpdate())
-          .build();
-    }
-
-    /**
-     * Returns an {@code Iterable<WindowedValue<InputT>>} that only contains non-late input
-     * elements.
-     */
-    Iterable<WindowedValue<V>> dropExpiredWindows(
-        final K key, Iterable<WindowedValue<V>> elements, final TimerInternals timerInternals) {
-      return StreamSupport.stream(elements.spliterator(), false)
-          .flatMap(wv -> StreamSupport.stream(wv.explodeWindows().spliterator(), false))
-          .filter(
-              input -> {
-                BoundedWindow window = getOnlyElement(input.getWindows());
-                boolean expired =
-                    window
-                        .maxTimestamp()
-                        .plus(windowingStrategy.getAllowedLateness())
-                        .isBefore(timerInternals.currentInputWatermarkTime());
-                if (expired) {
-                  // The element is too late for this window.
-                  droppedDueToLateness.inc();
-                  WindowTracing.debug(
-                      "{}: Dropping element at {} for key: {}; "
-                          + "window: {} since it is too far behind inputWatermark: {}",
-                      DirectGroupAlsoByWindow.class.getSimpleName(),
-                      input.getTimestamp(),
-                      key,
-                      window,
-                      timerInternals.currentInputWatermarkTime());
-                }
-                // Keep the element if the window is not expired.
-                return !expired;
-              })
-          .collect(Collectors.toList());
-    }
-  }
-
-  private static class OutputWindowedValueToBundle<K, V>
-      implements OutputWindowedValue<KV<K, Iterable<V>>> {
-    private final UncommittedBundle<KV<K, Iterable<V>>> bundle;
-
-    private OutputWindowedValueToBundle(UncommittedBundle<KV<K, Iterable<V>>> bundle) {
-      this.bundle = bundle;
-    }
-
-    @Override
-    public void outputWindowedValue(
-        KV<K, Iterable<V>> output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      bundle.add(WindowedValue.of(output, timestamp, windows, pane));
-    }
-
-    @Override
-    public <AdditionalOutputT> void outputWindowedValue(
-        TupleTag<AdditionalOutputT> tag,
-        AdditionalOutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      throw new UnsupportedOperationException(
-          String.format("%s should not use tagged outputs", "DirectGroupAlsoByWindow"));
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactory.java
deleted file mode 100644
index 46fee32..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactory.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components.Builder;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.construction.RehydratedComponents;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * The {@code DirectRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKeyOnly} {@link
- * PTransform}.
- */
-class GroupByKeyOnlyEvaluatorFactory implements TransformEvaluatorFactory {
-  private final Components components;
-
-  private final BundleFactory bundleFactory;
-  private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
-  GroupByKeyOnlyEvaluatorFactory(
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      Components components,
-      BundleFactory bundleFactory) {
-    this.components = components;
-    this.bundleFactory = bundleFactory;
-    this.graph = graph;
-  }
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator = (TransformEvaluator) createEvaluator(application);
-    return evaluator;
-  }
-
-  @Override
-  public void cleanup() {}
-
-  private <K, V> TransformEvaluator<KV<K, V>> createEvaluator(final PTransformNode application) {
-    return new GroupByKeyOnlyEvaluator<>(application);
-  }
-
-  /**
-   * A transform evaluator for the pseudo-primitive {@link GroupByKeyOnly}. Windowing is ignored;
-   * all input should be in the global window since all output will be as well.
-   *
-   * @see GroupByKeyViaGroupByKeyOnly
-   */
-  private class GroupByKeyOnlyEvaluator<K, V> implements TransformEvaluator<KV<K, V>> {
-    private final Coder<K> keyCoder;
-    private final Map<StructuralKey<K>, List<WindowedValue<V>>> groupingMap;
-
-    private final PCollectionNode outputPCollection;
-    private final StepTransformResult.Builder<KV<K, V>> resultBuilder;
-
-    private GroupByKeyOnlyEvaluator(PTransformNode application) {
-      keyCoder = getKeyCoder(application);
-      groupingMap = new HashMap<>();
-      outputPCollection = getOnlyElement(graph.getProduced(application));
-      resultBuilder = StepTransformResult.withoutHold(application);
-    }
-
-    private Coder<K> getKeyCoder(PTransformNode application) {
-      PCollectionNode inputPCollection = getOnlyElement(graph.getPerElementInputs(application));
-      try {
-        // We know the type restrictions on the input PCollection, and the restrictions on the
-        // Wire coder
-        Builder builder = GroupByKeyOnlyEvaluatorFactory.this.components.toBuilder();
-        String wireCoderId = WireCoders.addRunnerWireCoder(inputPCollection, builder);
-        Coder<WindowedValue<KV<K, V>>> wireCoder =
-            (Coder<WindowedValue<KV<K, V>>>)
-                RehydratedComponents.forComponents(builder.build()).getCoder(wireCoderId);
-
-        checkArgument(
-            wireCoder instanceof WindowedValue.WindowedValueCoder,
-            "Wire %s must be a %s",
-            Coder.class.getSimpleName(),
-            WindowedValueCoder.class.getSimpleName());
-        WindowedValueCoder<KV<K, V>> windowedValueCoder = (WindowedValueCoder<KV<K, V>>) wireCoder;
-
-        checkArgument(
-            windowedValueCoder.getValueCoder() instanceof KvCoder,
-            "Input elements to %s must be encoded with a %s",
-            DirectGroupByKey.DirectGroupByKeyOnly.class.getSimpleName(),
-            KvCoder.class.getSimpleName());
-        KvCoder<K, V> kvCoder = (KvCoder<K, V>) windowedValueCoder.getValueCoder();
-
-        return kvCoder.getKeyCoder();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override
-    public void processElement(WindowedValue<KV<K, V>> element) {
-      KV<K, V> kv = element.getValue();
-      K key = kv.getKey();
-      StructuralKey<K> groupingKey = StructuralKey.of(key, keyCoder);
-      List<WindowedValue<V>> values =
-          groupingMap.computeIfAbsent(groupingKey, k -> new ArrayList<>());
-      values.add(element.withValue(kv.getValue()));
-    }
-
-    @Override
-    public TransformResult<KV<K, V>> finishBundle() {
-      for (Map.Entry<StructuralKey<K>, List<WindowedValue<V>>> groupedEntry :
-          groupingMap.entrySet()) {
-        K key = groupedEntry.getKey().getKey();
-        KeyedWorkItem<K, V> groupedKv =
-            KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
-        UncommittedBundle<KeyedWorkItem<K, V>> bundle =
-            bundleFactory.createKeyedBundle(StructuralKey.of(key, keyCoder), outputPCollection);
-        bundle.add(WindowedValue.valueInGlobalWindow(groupedKv));
-        resultBuilder.addOutput(bundle);
-      }
-      return resultBuilder.build();
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImmutableListBundleFactory.java
deleted file mode 100644
index 1531e25..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImmutableListBundleFactory.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import com.google.auto.value.AutoValue;
-import java.util.Iterator;
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.MoreObjects;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.joda.time.Instant;
-
-/** A factory that produces bundles that perform no additional validation. */
-class ImmutableListBundleFactory implements BundleFactory {
-  private static final ImmutableListBundleFactory FACTORY = new ImmutableListBundleFactory();
-
-  public static ImmutableListBundleFactory create() {
-    return FACTORY;
-  }
-
-  private ImmutableListBundleFactory() {}
-
-  @Override
-  public <T> UncommittedBundle<T> createRootBundle() {
-    return UncommittedImmutableListBundle.create(null, StructuralKey.empty());
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createBundle(PCollectionNode output) {
-    return UncommittedImmutableListBundle.create(output, StructuralKey.empty());
-  }
-
-  @Override
-  public <K, T> UncommittedBundle<T> createKeyedBundle(
-      StructuralKey<K> key, PCollectionNode output) {
-    return UncommittedImmutableListBundle.create(output, key);
-  }
-
-  /** A {@link UncommittedBundle} that buffers elements in memory. */
-  private static final class UncommittedImmutableListBundle<T> implements UncommittedBundle<T> {
-    private final PCollectionNode pcollection;
-    private final StructuralKey<?> key;
-    private boolean committed = false;
-    private ImmutableList.Builder<WindowedValue<T>> elements;
-    private Instant minSoFar = BoundedWindow.TIMESTAMP_MAX_VALUE;
-
-    /**
-     * Create a new {@link UncommittedImmutableListBundle} for the specified {@link PCollection}.
-     */
-    public static <T> UncommittedImmutableListBundle<T> create(
-        PCollectionNode pcollection, StructuralKey<?> key) {
-      return new UncommittedImmutableListBundle<>(pcollection, key);
-    }
-
-    private UncommittedImmutableListBundle(PCollectionNode pcollection, StructuralKey<?> key) {
-      this.pcollection = pcollection;
-      this.key = key;
-      this.elements = ImmutableList.builder();
-    }
-
-    @Override
-    public PCollectionNode getPCollection() {
-      return pcollection;
-    }
-
-    @Override
-    public UncommittedImmutableListBundle<T> add(WindowedValue<T> element) {
-      checkState(
-          !committed,
-          "Can't add element %s to committed bundle in PCollection %s",
-          element,
-          pcollection);
-      checkArgument(
-          element.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE),
-          "Can't add an element past the end of time (%s), got timestamp %s",
-          BoundedWindow.TIMESTAMP_MAX_VALUE,
-          element.getTimestamp());
-      elements.add(element);
-      if (element.getTimestamp().isBefore(minSoFar)) {
-        minSoFar = element.getTimestamp();
-      }
-      return this;
-    }
-
-    @Override
-    public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
-      checkState(!committed, "Can't commit already committed bundle %s", this);
-      committed = true;
-      final Iterable<WindowedValue<T>> committedElements = elements.build();
-      return CommittedImmutableListBundle.create(
-          pcollection, key, committedElements, minSoFar, synchronizedCompletionTime);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this).add("elements", elements.build()).toString();
-    }
-  }
-
-  @AutoValue
-  abstract static class CommittedImmutableListBundle<T> implements CommittedBundle<T> {
-    public static <T> CommittedImmutableListBundle<T> create(
-        @Nullable PCollectionNode pcollection,
-        StructuralKey<?> key,
-        Iterable<WindowedValue<T>> committedElements,
-        Instant minElementTimestamp,
-        Instant synchronizedCompletionTime) {
-      return new AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle<>(
-          pcollection, key, committedElements, minElementTimestamp, synchronizedCompletionTime);
-    }
-
-    @Override
-    @Nonnull
-    public Iterator<WindowedValue<T>> iterator() {
-      return getElements().iterator();
-    }
-
-    @Override
-    public CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements) {
-      return create(
-          getPCollection(),
-          getKey(),
-          ImmutableList.copyOf(elements),
-          minTimestamp(elements),
-          getSynchronizedProcessingOutputWatermark());
-    }
-
-    @Override
-    public int hashCode() {
-      return System.identityHashCode(this);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return this == obj;
-    }
-  }
-
-  private static Instant minTimestamp(Iterable<? extends WindowedValue<?>> elements) {
-    Instant minTs = BoundedWindow.TIMESTAMP_MAX_VALUE;
-    for (WindowedValue<?> element : elements) {
-      if (element.getTimestamp().isBefore(minTs)) {
-        minTs = element.getTimestamp();
-      }
-    }
-    return minTs;
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImpulseEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImpulseEvaluatorFactory.java
deleted file mode 100644
index c2997ef..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImpulseEvaluatorFactory.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.util.Collection;
-import java.util.Collections;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.sdk.transforms.Impulse;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-
-/** The evaluator for the {@link Impulse} transform. Produces only empty byte arrays. */
-class ImpulseEvaluatorFactory implements TransformEvaluatorFactory {
-  private final BundleFactory bundleFactory;
-  private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
-  ImpulseEvaluatorFactory(
-      ExecutableGraph<PTransformNode, PCollectionNode> graph, BundleFactory bundleFactory) {
-    this.bundleFactory = bundleFactory;
-    this.graph = graph;
-  }
-
-  @Nullable
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) {
-    return (TransformEvaluator<InputT>)
-        new ImpulseEvaluator(
-            bundleFactory, application, getOnlyElement(graph.getProduced(application)));
-  }
-
-  @Override
-  public void cleanup() {
-    // Impulse has no state, so do nothing.
-  }
-
-  private static class ImpulseEvaluator implements TransformEvaluator<ImpulseShard> {
-    private final StepTransformResult.Builder<ImpulseShard> result;
-
-    private final BundleFactory factory;
-    private final PCollectionNode outputPCollection;
-
-    private ImpulseEvaluator(
-        BundleFactory factory, PTransformNode application, PCollectionNode outputPCollection) {
-      this.factory = factory;
-      result = StepTransformResult.withoutHold(application);
-      this.outputPCollection = outputPCollection;
-    }
-
-    @Override
-    public void processElement(WindowedValue<ImpulseShard> element) throws Exception {
-      result.addOutput(
-          factory
-              .createBundle(outputPCollection)
-              .add(WindowedValue.valueInGlobalWindow(new byte[0])));
-    }
-
-    @Override
-    public TransformResult<ImpulseShard> finishBundle() throws Exception {
-      return result.build();
-    }
-  }
-
-  /**
-   * The {@link RootInputProvider} for the {@link Impulse} {@link PTransform}. Produces a single
-   * {@link ImpulseShard}.
-   */
-  static class ImpulseRootProvider implements RootInputProvider<ImpulseShard> {
-    private final BundleFactory bundleFactory;
-
-    ImpulseRootProvider(BundleFactory bundleFactory) {
-      this.bundleFactory = bundleFactory;
-    }
-
-    @Override
-    public Collection<CommittedBundle<ImpulseShard>> getInitialInputs(
-        PTransformNode transform, int targetParallelism) {
-      return Collections.singleton(
-          bundleFactory
-              .<ImpulseShard>createRootBundle()
-              .add(WindowedValue.valueInGlobalWindow(new ImpulseShard()))
-              .commit(BoundedWindow.TIMESTAMP_MIN_VALUE));
-    }
-  }
-
-  @VisibleForTesting
-  static class ImpulseShard {}
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWindow.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWindow.java
deleted file mode 100644
index a632daf..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWindow.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Objects;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.PCollectionView;
-
-/**
- * A pair of a {@link PCollectionView} and a {@link BoundedWindow}, which can be thought of as
- * window "of" the view. This is a value class for use e.g. as a compound cache key.
- *
- * @param <T> the type of the underlying PCollectionView
- */
-final class PCollectionViewWindow<T> {
-
-  private final PCollectionView<T> view;
-  private final BoundedWindow window;
-
-  private PCollectionViewWindow(PCollectionView<T> view, BoundedWindow window) {
-    this.view = view;
-    this.window = window;
-  }
-
-  public static <T> PCollectionViewWindow<T> of(PCollectionView<T> view, BoundedWindow window) {
-    return new PCollectionViewWindow<>(view, window);
-  }
-
-  public PCollectionView<T> getView() {
-    return view;
-  }
-
-  public BoundedWindow getWindow() {
-    return window;
-  }
-
-  @Override
-  public boolean equals(Object otherObject) {
-    if (!(otherObject instanceof PCollectionViewWindow)) {
-      return false;
-    }
-    @SuppressWarnings("unchecked")
-    PCollectionViewWindow<T> other = (PCollectionViewWindow<T>) otherObject;
-    return getView().equals(other.getView()) && getWindow().equals(other.getWindow());
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(getView(), getWindow());
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWriter.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWriter.java
deleted file mode 100644
index 944caca..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWriter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-/**
- * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to a
- * storage mechanism that can be read from while constructing a {@link PCollectionView}.
- *
- * @param <ElemT> the type of elements the input {@link PCollection} contains.
- * @param <ViewT> the type of the PCollectionView this writer writes to.
- */
-@FunctionalInterface
-interface PCollectionViewWriter<ElemT, ViewT> {
-  void add(Iterable<WindowedValue<ElemT>> values);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PassthroughTransformEvaluator.java
deleted file mode 100644
index 831bb93..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PassthroughTransformEvaluator.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.util.WindowedValue;
-
-class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
-  public static <InputT> PassthroughTransformEvaluator<InputT> create(
-      PTransformNode transform, UncommittedBundle<InputT> output) {
-    return new PassthroughTransformEvaluator<>(transform, output);
-  }
-
-  private final PTransformNode transform;
-  private final UncommittedBundle<InputT> output;
-
-  private PassthroughTransformEvaluator(
-      PTransformNode transform, UncommittedBundle<InputT> output) {
-    this.transform = transform;
-    this.output = output;
-  }
-
-  @Override
-  public void processElement(WindowedValue<InputT> element) throws Exception {
-    output.add(element);
-  }
-
-  @Override
-  public TransformResult<InputT> finishBundle() throws Exception {
-    return StepTransformResult.<InputT>withoutHold(transform).addOutput(output).build();
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PipelineExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PipelineExecutor.java
deleted file mode 100644
index f169f3e..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PipelineExecutor.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.joda.time.Duration;
-
-/**
- * An executor that schedules and executes {@link PTransformNode PTransformNodes} for both source
- * and intermediate {@link PTransform PTransforms}.
- */
-interface PipelineExecutor {
-  /**
-   * Starts this executor on the provided graph. The {@link RootProviderRegistry} will be used to
-   * create initial inputs for the provide {@link ExecutableGraph graph}.
-   */
-  void start();
-
-  /**
-   * Blocks until the job being executed enters a terminal state. A job is completed after all root
-   * {@link PTransformNode PTransformNodes} have completed, and all {@link CommittedBundle Bundles}
-   * have been consumed. Jobs may also terminate abnormally.
-   *
-   * <p>Waits for up to the provided duration, or forever if the provided duration is less than or
-   * equal to zero.
-   *
-   * @return The terminal state of the Pipeline.
-   * @throws Exception whenever an executor thread throws anything, transfers to the waiting thread
-   *     and rethrows it
-   */
-  State waitUntilFinish(Duration duration) throws Exception;
-
-  /** Gets the current state of the {@link Pipeline}. */
-  State getPipelineState();
-
-  /**
-   * Shuts down the executor.
-   *
-   * <p>The executor may continue to run for a short time after this method returns.
-   */
-  void stop();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PortableGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PortableGraph.java
deleted file mode 100644
index 88b6e8c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PortableGraph.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
-import org.apache.beam.runners.direct.ExecutableGraph;
-
-/** A {@link ExecutableGraph} for a Portable {@link RunnerApi.Pipeline}. */
-class PortableGraph implements ExecutableGraph<PTransformNode, PCollectionNode> {
-  private final QueryablePipeline queryablePipeline;
-
-  public static PortableGraph forPipeline(RunnerApi.Pipeline p) {
-    return new PortableGraph(p);
-  }
-
-  private PortableGraph(RunnerApi.Pipeline p) {
-    this.queryablePipeline =
-        QueryablePipeline.forTransforms(p.getRootTransformIdsList(), p.getComponents());
-  }
-
-  @Override
-  public Collection<PTransformNode> getRootTransforms() {
-    return queryablePipeline.getRootTransforms();
-  }
-
-  @Override
-  public Collection<PTransformNode> getExecutables() {
-    return queryablePipeline.getTransforms();
-  }
-
-  @Override
-  public PTransformNode getProducer(PCollectionNode collection) {
-    return queryablePipeline.getProducer(collection);
-  }
-
-  @Override
-  public Collection<PCollectionNode> getProduced(PTransformNode producer) {
-    return queryablePipeline.getOutputPCollections(producer);
-  }
-
-  @Override
-  public Collection<PCollectionNode> getPerElementInputs(PTransformNode transform) {
-    return queryablePipeline.getPerElementInputPCollections(transform);
-  }
-
-  @Override
-  public Collection<PTransformNode> getPerElementConsumers(PCollectionNode pCollection) {
-    return queryablePipeline.getPerElementConsumers(pCollection);
-  }
-
-  public QueryablePipeline getQueryablePipeline() {
-    return this.queryablePipeline;
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/QuiescenceDriver.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/QuiescenceDriver.java
deleted file mode 100644
index b11bc91..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/QuiescenceDriver.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import com.google.auto.value.AutoValue;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
-import org.apache.beam.runners.local.ExecutionDriver;
-import org.apache.beam.runners.local.PipelineMessageReceiver;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Pushes additional work onto a {@link BundleProcessor} based on the fact that a pipeline has
- * quiesced.
- */
-class QuiescenceDriver implements ExecutionDriver {
-  private static final Logger LOG = LoggerFactory.getLogger(QuiescenceDriver.class);
-
-  public static ExecutionDriver create(
-      EvaluationContext context,
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode> bundleProcessor,
-      PipelineMessageReceiver messageReceiver,
-      Map<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> initialBundles) {
-    return new QuiescenceDriver(context, graph, bundleProcessor, messageReceiver, initialBundles);
-  }
-
-  private final EvaluationContext evaluationContext;
-  private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-  private final BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode>
-      bundleProcessor;
-  private final PipelineMessageReceiver pipelineMessageReceiver;
-
-  private final CompletionCallback defaultCompletionCallback =
-      new TimerIterableCompletionCallback(Collections.emptyList());
-
-  private final Map<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> pendingRootBundles;
-  private final Queue<WorkUpdate> pendingWork = new ConcurrentLinkedQueue<>();
-
-  private final AtomicReference<ExecutorState> state =
-      new AtomicReference<>(ExecutorState.QUIESCENT);
-  private final AtomicLong outstandingWork = new AtomicLong(0L);
-  private boolean exceptionThrown = false;
-
-  private QuiescenceDriver(
-      EvaluationContext evaluationContext,
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode> bundleProcessor,
-      PipelineMessageReceiver pipelineMessageReceiver,
-      Map<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> pendingRootBundles) {
-    this.evaluationContext = evaluationContext;
-    this.graph = graph;
-    this.bundleProcessor = bundleProcessor;
-    this.pipelineMessageReceiver = pipelineMessageReceiver;
-    this.pendingRootBundles = pendingRootBundles;
-  }
-
-  @Override
-  public DriverState drive() {
-    boolean noWorkOutstanding = outstandingWork.get() == 0L;
-    ExecutorState startingState = state.get();
-    if (startingState == ExecutorState.ACTIVE) {
-      // The remainder of this call will add all available work to the Executor, and there will
-      // be no new work available
-      state.compareAndSet(ExecutorState.ACTIVE, ExecutorState.PROCESSING);
-    } else if (startingState == ExecutorState.PROCESSING && noWorkOutstanding) {
-      // The executor has consumed all new work and no new work was added
-      state.compareAndSet(ExecutorState.PROCESSING, ExecutorState.QUIESCING);
-    } else if (startingState == ExecutorState.QUIESCING && noWorkOutstanding) {
-      // The executor re-ran all blocked work and nothing could make progress.
-      state.compareAndSet(ExecutorState.QUIESCING, ExecutorState.QUIESCENT);
-    }
-    fireTimers();
-    Collection<WorkUpdate> updates = new ArrayList<>();
-    // Pull all available updates off of the queue before adding additional work. This ensures
-    // both loops terminate.
-    WorkUpdate pendingUpdate = pendingWork.poll();
-    while (pendingUpdate != null) {
-      updates.add(pendingUpdate);
-      pendingUpdate = pendingWork.poll();
-    }
-    for (WorkUpdate update : updates) {
-      applyUpdate(noWorkOutstanding, startingState, update);
-    }
-    addWorkIfNecessary();
-
-    if (exceptionThrown) {
-      return DriverState.FAILED;
-    } else if (evaluationContext.isDone()) {
-      return DriverState.SHUTDOWN;
-    } else {
-      return DriverState.CONTINUE;
-    }
-  }
-
-  private void applyUpdate(
-      boolean noWorkOutstanding, ExecutorState startingState, WorkUpdate update) {
-    LOG.debug("Executor Update: {}", update);
-    if (update.getBundle().isPresent()) {
-      if (ExecutorState.ACTIVE == startingState
-          || (ExecutorState.PROCESSING == startingState && noWorkOutstanding)) {
-        CommittedBundle<?> bundle = update.getBundle().get();
-        for (PTransformNode consumer : update.getConsumers()) {
-          outstandingWork.incrementAndGet();
-          bundleProcessor.process(bundle, consumer, defaultCompletionCallback);
-        }
-      } else {
-        pendingWork.offer(update);
-      }
-    } else if (update.getException().isPresent()) {
-      pipelineMessageReceiver.failed(update.getException().get());
-      exceptionThrown = true;
-    }
-  }
-
-  /** Fires any available timers. */
-  private void fireTimers() {
-    try {
-      for (FiredTimers<PTransformNode> transformTimers : evaluationContext.extractFiredTimers()) {
-        Collection<TimerData> delivery = transformTimers.getTimers();
-        KeyedWorkItem<?, Object> work =
-            KeyedWorkItems.timersWorkItem(transformTimers.getKey().getKey(), delivery);
-        PCollectionNode inputPCollection =
-            Iterables.getOnlyElement(graph.getPerElementInputs(transformTimers.getExecutable()));
-        @SuppressWarnings({"unchecked", "rawtypes"})
-        CommittedBundle<?> bundle =
-            evaluationContext
-                .createKeyedBundle(transformTimers.getKey(), inputPCollection)
-                .add(WindowedValue.valueInGlobalWindow(work))
-                .commit(evaluationContext.now());
-        outstandingWork.incrementAndGet();
-        bundleProcessor.process(
-            bundle, transformTimers.getExecutable(), new TimerIterableCompletionCallback(delivery));
-        state.set(ExecutorState.ACTIVE);
-      }
-    } catch (Exception e) {
-      LOG.error("Internal Error while delivering timers", e);
-      pipelineMessageReceiver.failed(e);
-      exceptionThrown = true;
-    }
-  }
-
-  /**
-   * If all active {@link DirectTransformExecutor TransformExecutors} are in a blocked state, add
-   * more work from root nodes that may have additional work. This ensures that if a pipeline has
-   * elements available from the root nodes it will add those elements when necessary.
-   */
-  private void addWorkIfNecessary() {
-    // If any timers have fired, they will add more work; We don't need to add more
-    if (state.get() == ExecutorState.QUIESCENT) {
-      // All current TransformExecutors are blocked; add more work from the roots.
-      for (Map.Entry<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> pendingRootEntry :
-          pendingRootBundles.entrySet()) {
-        Collection<CommittedBundle<?>> bundles = new ArrayList<>();
-        // Pull all available work off of the queue, then schedule it all, so this loop
-        // terminates
-        while (!pendingRootEntry.getValue().isEmpty()) {
-          CommittedBundle<?> bundle = pendingRootEntry.getValue().poll();
-          bundles.add(bundle);
-        }
-        for (CommittedBundle<?> bundle : bundles) {
-          outstandingWork.incrementAndGet();
-          bundleProcessor.process(bundle, pendingRootEntry.getKey(), defaultCompletionCallback);
-          state.set(ExecutorState.ACTIVE);
-        }
-      }
-    }
-  }
-
-  /**
-   * The state of the executor. The state of the executor determines the behavior of the {@link
-   * QuiescenceDriver} when it runs.
-   */
-  private enum ExecutorState {
-    /**
-     * Output has been produced since the last time the monitor ran. Work exists that has not yet
-     * been evaluated, and all pending, including potentially blocked work, should be evaluated.
-     *
-     * <p>The executor becomes active whenever a timer fires, a {@link PCollectionView} is updated,
-     * or output is produced by the evaluation of a {@link DirectTransformExecutor}.
-     */
-    ACTIVE,
-    /**
-     * The Executor does not have any unevaluated work available to it, but work is in progress.
-     * Work should not be added until the Executor becomes active or no work is outstanding.
-     *
-     * <p>If all outstanding work completes without the executor becoming {@code ACTIVE}, the
-     * Executor enters state {@code QUIESCING}. Previously evaluated work must be reevaluated, in
-     * case a side input has made progress.
-     */
-    PROCESSING,
-    /**
-     * All outstanding work is work that may be blocked on a side input. When there is no
-     * outstanding work, the executor becomes {@code QUIESCENT}.
-     */
-    QUIESCING,
-    /**
-     * All elements are either buffered in state or are blocked on a side input. There are no timers
-     * that are permitted to fire but have not. There is no outstanding work.
-     *
-     * <p>The pipeline will not make progress without the progression of watermarks, the progression
-     * of processing time, or the addition of elements.
-     */
-    QUIESCENT
-  }
-
-  /**
-   * The base implementation of {@link CompletionCallback} that provides implementations for {@link
-   * #handleResult(CommittedBundle, TransformResult)} and {@link #handleException(CommittedBundle,
-   * Exception)}.
-   */
-  private class TimerIterableCompletionCallback implements CompletionCallback {
-    private final Iterable<TimerData> timers;
-
-    TimerIterableCompletionCallback(Iterable<TimerData> timers) {
-      this.timers = timers;
-    }
-
-    @Override
-    public final CommittedResult handleResult(
-        CommittedBundle<?> inputBundle, TransformResult<?> result) {
-      CommittedResult<PTransformNode> committedResult =
-          evaluationContext.handleResult(inputBundle, timers, result);
-      for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
-        pendingWork.offer(
-            WorkUpdate.fromBundle(
-                outputBundle, graph.getPerElementConsumers(outputBundle.getPCollection())));
-      }
-      Optional<? extends CommittedBundle<?>> unprocessedInputs =
-          committedResult.getUnprocessedInputs();
-      if (unprocessedInputs.isPresent()) {
-        if (inputBundle.getPCollection() == null) {
-          // TODO: Split this logic out of an if statement
-          pendingRootBundles.get(result.getTransform()).offer(unprocessedInputs.get());
-        } else {
-          pendingWork.offer(
-              WorkUpdate.fromBundle(
-                  unprocessedInputs.get(), Collections.singleton(committedResult.getExecutable())));
-        }
-      }
-      if (!committedResult.getProducedOutputTypes().isEmpty()) {
-        state.set(ExecutorState.ACTIVE);
-      }
-      outstandingWork.decrementAndGet();
-      return committedResult;
-    }
-
-    @Override
-    public void handleEmpty(PTransformNode transform) {
-      outstandingWork.decrementAndGet();
-    }
-
-    @Override
-    public final void handleException(CommittedBundle<?> inputBundle, Exception e) {
-      pendingWork.offer(WorkUpdate.fromException(e));
-      outstandingWork.decrementAndGet();
-    }
-
-    @Override
-    public void handleError(Error err) {
-      outstandingWork.decrementAndGet();
-      pipelineMessageReceiver.failed(err);
-    }
-  }
-
-  /**
-   * An internal status update on the state of the executor.
-   *
-   * <p>Used to signal when the executor should be shut down (due to an exception).
-   */
-  @AutoValue
-  abstract static class WorkUpdate {
-    private static WorkUpdate fromBundle(
-        CommittedBundle<?> bundle, Collection<PTransformNode> consumers) {
-      return new AutoValue_QuiescenceDriver_WorkUpdate(
-          Optional.of(bundle), consumers, Optional.absent());
-    }
-
-    private static WorkUpdate fromException(Exception e) {
-      return new AutoValue_QuiescenceDriver_WorkUpdate(
-          Optional.absent(), Collections.emptyList(), Optional.of(e));
-    }
-
-    /** Returns the bundle that produced this update. */
-    public abstract Optional<? extends CommittedBundle<?>> getBundle();
-
-    /**
-     * Returns the transforms to process the bundle. If nonempty, {@link #getBundle()} will return a
-     * present {@link Optional}.
-     */
-    public abstract Collection<PTransformNode> getConsumers();
-
-    public abstract Optional<? extends Exception> getException();
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ReferenceRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ReferenceRunner.java
deleted file mode 100644
index eee804c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ReferenceRunner.java
+++ /dev/null
@@ -1,559 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.runners.core.construction.SyntheticComponents.uniqueId;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.stream.Collectors;
-import org.apache.beam.model.fnexecution.v1.ProvisionApi.ProvisionInfo;
-import org.apache.beam.model.fnexecution.v1.ProvisionApi.Resources;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Coder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ComponentsOrBuilder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform.Builder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
-import org.apache.beam.runners.core.construction.ModelCoders;
-import org.apache.beam.runners.core.construction.ModelCoders.KvCoderComponents;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.construction.graph.PipelineValidator;
-import org.apache.beam.runners.core.construction.graph.ProtoOverrides;
-import org.apache.beam.runners.core.construction.graph.ProtoOverrides.TransformReplacement;
-import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.fnexecution.GrpcContextHeaderAccessorProvider;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.InProcessServerFactory;
-import org.apache.beam.runners.fnexecution.ServerFactory;
-import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
-import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactRetrievalService;
-import org.apache.beam.runners.fnexecution.control.ControlClientPool;
-import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.runners.fnexecution.control.MapControlClientPool;
-import org.apache.beam.runners.fnexecution.control.SingleEnvironmentInstanceJobBundleFactory;
-import org.apache.beam.runners.fnexecution.data.GrpcDataService;
-import org.apache.beam.runners.fnexecution.environment.DockerEnvironmentFactory;
-import org.apache.beam.runners.fnexecution.environment.EmbeddedEnvironmentFactory;
-import org.apache.beam.runners.fnexecution.environment.EnvironmentFactory;
-import org.apache.beam.runners.fnexecution.logging.GrpcLoggingService;
-import org.apache.beam.runners.fnexecution.logging.Slf4jLogWriter;
-import org.apache.beam.runners.fnexecution.provisioning.StaticGrpcProvisionService;
-import org.apache.beam.runners.fnexecution.state.GrpcStateService;
-import org.apache.beam.runners.fnexecution.wire.LengthPrefixUnknownCoders;
-import org.apache.beam.sdk.fn.IdGenerator;
-import org.apache.beam.sdk.fn.IdGenerators;
-import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.Struct;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Maps;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Sets;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * The "ReferenceRunner" engine implementation. The ReferenceRunner uses the portability framework
- * to execute a Pipeline on a single machine.
- */
-public class ReferenceRunner {
-  private final RunnerApi.Pipeline pipeline;
-  private final Struct options;
-  private final String artifactRetrievalToken;
-
-  private final EnvironmentType environmentType;
-
-  private final IdGenerator idGenerator = IdGenerators.incrementingLongs();
-
-  /** @param environmentType The environment to use for the SDK Harness. */
-  private ReferenceRunner(
-      Pipeline p, Struct options, String artifactRetrievalToken, EnvironmentType environmentType) {
-    this.pipeline = executable(p);
-    this.options = options;
-    this.environmentType = environmentType;
-    this.artifactRetrievalToken = artifactRetrievalToken;
-  }
-
-  /**
-   * Creates a "ReferenceRunner" engine for a single pipeline with a Dockerized SDK harness.
-   *
-   * @param p Pipeline being executed for this job.
-   * @param options PipelineOptions for this job.
-   * @param artifactRetrievalToken Token to retrieve artifacts that have been staged.
-   */
-  public static ReferenceRunner forPipeline(
-      RunnerApi.Pipeline p, Struct options, String artifactRetrievalToken) {
-    return new ReferenceRunner(p, options, artifactRetrievalToken, EnvironmentType.DOCKER);
-  }
-
-  static ReferenceRunner forInProcessPipeline(RunnerApi.Pipeline p, Struct options) {
-    return new ReferenceRunner(p, options, "", EnvironmentType.IN_PROCESS);
-  }
-
-  private RunnerApi.Pipeline executable(RunnerApi.Pipeline original) {
-    RunnerApi.Pipeline p = original;
-    PipelineValidator.validate(p);
-    p =
-        ProtoOverrides.updateTransform(
-            PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN,
-            p,
-            new SplittableProcessKeyedReplacer());
-    p =
-        ProtoOverrides.updateTransform(
-            PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, p, new PortableGroupByKeyReplacer());
-    p = GreedyPipelineFuser.fuse(p).toPipeline();
-
-    p = foldFeedSDFIntoExecutableStage(p);
-    PipelineValidator.validate(p);
-
-    return p;
-  }
-
-  private static Set<PCollectionNode> getKeyedPCollections(
-      ExecutableGraph<PTransformNode, PCollectionNode> graph) {
-    // This mimics KeyedPValueTrackingVisitor behavior in regular direct runner,
-    // but without propagating keyed-ness through key-preserving DoFn's.
-    // That is not yet necessary, but will be necessary once we implement state and timers.
-    // See https://issues.apache.org/jira/browse/BEAM-4557.
-    Set<PCollectionNode> res = Sets.newHashSet();
-    Set<String> keyedProducers =
-        Sets.newHashSet(DirectGroupByKey.DIRECT_GBKO_URN, DirectGroupByKey.DIRECT_GABW_URN);
-    for (PTransformNode transform : graph.getExecutables()) {
-      if (keyedProducers.contains(transform.getTransform().getSpec().getUrn())) {
-        res.addAll(graph.getProduced(transform));
-      }
-    }
-    return res;
-  }
-
-  /**
-   * First starts all the services needed, then configures and starts the {@link
-   * ExecutorServiceParallelExecutor}.
-   */
-  public void execute() throws Exception {
-    ExecutableGraph<PTransformNode, PCollectionNode> graph = PortableGraph.forPipeline(pipeline);
-    BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-    EvaluationContext ctxt =
-        EvaluationContext.create(Instant::new, bundleFactory, graph, getKeyedPCollections(graph));
-    RootProviderRegistry rootRegistry = RootProviderRegistry.javaPortableRegistry(bundleFactory);
-    int targetParallelism = Math.max(Runtime.getRuntime().availableProcessors(), 3);
-    ServerFactory serverFactory = createServerFactory();
-    ControlClientPool controlClientPool = MapControlClientPool.create();
-    ExecutorService dataExecutor = Executors.newCachedThreadPool();
-    ProvisionInfo provisionInfo =
-        ProvisionInfo.newBuilder()
-            .setJobId("id")
-            .setJobName("reference")
-            .setPipelineOptions(options)
-            .setWorkerId("foo")
-            .setResourceLimits(Resources.getDefaultInstance())
-            .setRetrievalToken(artifactRetrievalToken)
-            .build();
-    try (GrpcFnServer<GrpcLoggingService> logging =
-            GrpcFnServer.allocatePortAndCreateFor(
-                GrpcLoggingService.forWriter(Slf4jLogWriter.getDefault()), serverFactory);
-        GrpcFnServer<ArtifactRetrievalService> artifact =
-            GrpcFnServer.allocatePortAndCreateFor(
-                BeamFileSystemArtifactRetrievalService.create(), serverFactory);
-        GrpcFnServer<StaticGrpcProvisionService> provisioning =
-            GrpcFnServer.allocatePortAndCreateFor(
-                StaticGrpcProvisionService.create(provisionInfo), serverFactory);
-        GrpcFnServer<FnApiControlClientPoolService> control =
-            GrpcFnServer.allocatePortAndCreateFor(
-                FnApiControlClientPoolService.offeringClientsToPool(
-                    controlClientPool.getSink(),
-                    GrpcContextHeaderAccessorProvider.getHeaderAccessor()),
-                serverFactory);
-        GrpcFnServer<GrpcDataService> data =
-            GrpcFnServer.allocatePortAndCreateFor(
-                GrpcDataService.create(dataExecutor, OutboundObserverFactory.serverDirect()),
-                serverFactory);
-        GrpcFnServer<GrpcStateService> state =
-            GrpcFnServer.allocatePortAndCreateFor(GrpcStateService.create(), serverFactory)) {
-
-      EnvironmentFactory environmentFactory =
-          createEnvironmentFactory(control, logging, artifact, provisioning, controlClientPool);
-      JobBundleFactory jobBundleFactory =
-          SingleEnvironmentInstanceJobBundleFactory.create(
-              environmentFactory, data, state, idGenerator);
-
-      TransformEvaluatorRegistry transformRegistry =
-          TransformEvaluatorRegistry.portableRegistry(
-              graph,
-              pipeline.getComponents(),
-              bundleFactory,
-              jobBundleFactory,
-              EvaluationContextStepStateAndTimersProvider.forContext(ctxt));
-      ExecutorServiceParallelExecutor executor =
-          ExecutorServiceParallelExecutor.create(
-              targetParallelism, rootRegistry, transformRegistry, graph, ctxt);
-      executor.start();
-      executor.waitUntilFinish(Duration.ZERO);
-    } finally {
-      dataExecutor.shutdown();
-    }
-  }
-
-  private ServerFactory createServerFactory() {
-    switch (environmentType) {
-      case DOCKER:
-        return ServerFactory.createDefault();
-      case IN_PROCESS:
-        return InProcessServerFactory.create();
-      default:
-        throw new IllegalArgumentException(
-            String.format("Unknown %s %s", EnvironmentType.class.getSimpleName(), environmentType));
-    }
-  }
-
-  private EnvironmentFactory createEnvironmentFactory(
-      GrpcFnServer<FnApiControlClientPoolService> control,
-      GrpcFnServer<GrpcLoggingService> logging,
-      GrpcFnServer<ArtifactRetrievalService> artifact,
-      GrpcFnServer<StaticGrpcProvisionService> provisioning,
-      ControlClientPool controlClient) {
-    switch (environmentType) {
-      case DOCKER:
-        return new DockerEnvironmentFactory.Provider(PipelineOptionsTranslation.fromProto(options))
-            .createEnvironmentFactory(
-                control, logging, artifact, provisioning, controlClient, idGenerator);
-      case IN_PROCESS:
-        return EmbeddedEnvironmentFactory.create(
-            PipelineOptionsFactory.create(), logging, control, controlClient.getSource());
-      default:
-        throw new IllegalArgumentException(
-            String.format("Unknown %s %s", EnvironmentType.class.getSimpleName(), environmentType));
-    }
-  }
-
-  @VisibleForTesting
-  static class PortableGroupByKeyReplacer implements TransformReplacement {
-    @Override
-    public MessageWithComponents getReplacement(String gbkId, ComponentsOrBuilder components) {
-      PTransform gbk = components.getTransformsOrThrow(gbkId);
-      checkArgument(
-          PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN.equals(gbk.getSpec().getUrn()),
-          "URN must be %s, got %s",
-          PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN,
-          gbk.getSpec().getUrn());
-
-      PTransform.Builder newTransform = gbk.toBuilder();
-      Components.Builder newComponents = Components.newBuilder();
-      String inputId = getOnlyElement(gbk.getInputsMap().values());
-
-      // Add the GBKO transform
-      String kwiCollectionId =
-          uniqueId(String.format("%s.%s", inputId, "kwi"), components::containsPcollections);
-      {
-        PCollection input = components.getPcollectionsOrThrow(inputId);
-        Coder inputCoder = components.getCodersOrThrow(input.getCoderId());
-        KvCoderComponents kvComponents = ModelCoders.getKvCoderComponents(inputCoder);
-        String windowCoderId =
-            components
-                .getWindowingStrategiesOrThrow(input.getWindowingStrategyId())
-                .getWindowCoderId();
-        // This coder isn't actually required for the pipeline to function properly - the KWIs can
-        // be passed around as pure java objects with no coding of the values, but it approximates
-        // a full pipeline.
-        Coder kwiCoder =
-            Coder.newBuilder()
-                .setSpec(
-                    SdkFunctionSpec.newBuilder()
-                        .setSpec(FunctionSpec.newBuilder().setUrn("beam:direct:keyedworkitem:v1")))
-                .addAllComponentCoderIds(
-                    ImmutableList.of(
-                        kvComponents.keyCoderId(), kvComponents.valueCoderId(), windowCoderId))
-                .build();
-        String kwiCoderId =
-            uniqueId(
-                String.format("kwi(%s:%s)", kvComponents.keyCoderId(), kvComponents.valueCoderId()),
-                components::containsCoders);
-        // The kwi PCollection has the same WindowingStrategy as the input, as no merging will
-        // have been performed, so elements remain in their original windows
-        PCollection kwi =
-            input.toBuilder().setUniqueName(kwiCollectionId).setCoderId(kwiCoderId).build();
-        String gbkoId = uniqueId(String.format("%s/GBKO", gbkId), components::containsTransforms);
-        PTransform gbko =
-            PTransform.newBuilder()
-                .setUniqueName(String.format("%s/GBKO", gbk.getUniqueName()))
-                .putAllInputs(gbk.getInputsMap())
-                .setSpec(FunctionSpec.newBuilder().setUrn(DirectGroupByKey.DIRECT_GBKO_URN))
-                .putOutputs("output", kwiCollectionId)
-                .build();
-
-        newTransform.addSubtransforms(gbkoId);
-        newComponents
-            .putCoders(kwiCoderId, kwiCoder)
-            .putPcollections(kwiCollectionId, kwi)
-            .putTransforms(gbkoId, gbko);
-      }
-
-      // Add the GABW transform
-      {
-        String gabwId = uniqueId(String.format("%s/GABW", gbkId), components::containsTransforms);
-        PTransform gabw =
-            PTransform.newBuilder()
-                .setUniqueName(String.format("%s/GABW", gbk.getUniqueName()))
-                .putInputs("input", kwiCollectionId)
-                .setSpec(FunctionSpec.newBuilder().setUrn(DirectGroupByKey.DIRECT_GABW_URN))
-                .putAllOutputs(gbk.getOutputsMap())
-                .build();
-        newTransform.addSubtransforms(gabwId);
-        newComponents.putTransforms(gabwId, gabw);
-      }
-
-      return MessageWithComponents.newBuilder()
-          .setPtransform(newTransform)
-          .setComponents(newComponents)
-          .build();
-    }
-  }
-
-  /**
-   * Replaces the {@link PTransformTranslation#SPLITTABLE_PROCESS_KEYED_URN} with a {@link
-   * DirectGroupByKey#DIRECT_GBKO_URN} (construct keyed work items) followed by a {@link
-   * SplittableRemoteStageEvaluatorFactory#FEED_SDF_URN} (convert the keyed work items to
-   * element/restriction pairs that later go into {@link
-   * PTransformTranslation#SPLITTABLE_PROCESS_ELEMENTS_URN}).
-   */
-  @VisibleForTesting
-  static class SplittableProcessKeyedReplacer implements TransformReplacement {
-    @Override
-    public MessageWithComponents getReplacement(String spkId, ComponentsOrBuilder components) {
-      PTransform spk = components.getTransformsOrThrow(spkId);
-      checkArgument(
-          PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN.equals(spk.getSpec().getUrn()),
-          "URN must be %s, got %s",
-          PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN,
-          spk.getSpec().getUrn());
-
-      Components.Builder newComponents = Components.newBuilder();
-      newComponents.putAllCoders(components.getCodersMap());
-
-      Builder newPTransform = spk.toBuilder();
-
-      String inputId = getOnlyElement(spk.getInputsMap().values());
-      PCollection input = components.getPcollectionsOrThrow(inputId);
-
-      // This is a Coder<KV<String, KV<ElementT, RestrictionT>>>
-      Coder inputCoder = components.getCodersOrThrow(input.getCoderId());
-      KvCoderComponents kvComponents = ModelCoders.getKvCoderComponents(inputCoder);
-      String windowCoderId =
-          components
-              .getWindowingStrategiesOrThrow(input.getWindowingStrategyId())
-              .getWindowCoderId();
-
-      // === Construct a raw GBK returning KeyedWorkItem's ===
-      String kwiCollectionId =
-          uniqueId(String.format("%s.kwi", spkId), components::containsPcollections);
-      {
-        // This coder isn't actually required for the pipeline to function properly - the KWIs can
-        // be passed around as pure java objects with no coding of the values, but it approximates a
-        // full pipeline.
-        Coder kwiCoder =
-            Coder.newBuilder()
-                .setSpec(
-                    SdkFunctionSpec.newBuilder()
-                        .setSpec(FunctionSpec.newBuilder().setUrn("beam:direct:keyedworkitem:v1")))
-                .addAllComponentCoderIds(
-                    ImmutableList.of(
-                        kvComponents.keyCoderId(), kvComponents.valueCoderId(), windowCoderId))
-                .build();
-        String kwiCoderId =
-            uniqueId(
-                String.format(
-                    "keyed_work_item(%s:%s)",
-                    kvComponents.keyCoderId(), kvComponents.valueCoderId()),
-                components::containsCoders);
-
-        PCollection kwiCollection =
-            input.toBuilder().setUniqueName(kwiCollectionId).setCoderId(kwiCoderId).build();
-        String rawGbkId =
-            uniqueId(String.format("%s/RawGBK", spkId), components::containsTransforms);
-        PTransform rawGbk =
-            PTransform.newBuilder()
-                .setUniqueName(String.format("%s/RawGBK", spk.getUniqueName()))
-                .putAllInputs(spk.getInputsMap())
-                .setSpec(FunctionSpec.newBuilder().setUrn(DirectGroupByKey.DIRECT_GBKO_URN))
-                .putOutputs("output", kwiCollectionId)
-                .build();
-
-        newComponents
-            .putCoders(kwiCoderId, kwiCoder)
-            .putPcollections(kwiCollectionId, kwiCollection)
-            .putTransforms(rawGbkId, rawGbk);
-        newPTransform.addSubtransforms(rawGbkId);
-      }
-
-      // === Construct a "Feed SDF" operation that converts KWI to KV<ElementT, RestrictionT> ===
-      String feedSDFCollectionId =
-          uniqueId(String.format("%s.feed", spkId), components::containsPcollections);
-      {
-        String elementRestrictionCoderId = kvComponents.valueCoderId();
-        String feedSDFCoderId =
-            LengthPrefixUnknownCoders.addLengthPrefixedCoder(
-                elementRestrictionCoderId, newComponents, false);
-
-        PCollection feedSDFCollection =
-            input.toBuilder().setUniqueName(feedSDFCollectionId).setCoderId(feedSDFCoderId).build();
-        String feedSDFId =
-            uniqueId(String.format("%s/FeedSDF", spkId), components::containsTransforms);
-        PTransform feedSDF =
-            PTransform.newBuilder()
-                .setUniqueName(String.format("%s/FeedSDF", spk.getUniqueName()))
-                .putInputs("input", kwiCollectionId)
-                .setSpec(
-                    FunctionSpec.newBuilder()
-                        .setUrn(SplittableRemoteStageEvaluatorFactory.FEED_SDF_URN))
-                .putOutputs("output", feedSDFCollectionId)
-                .build();
-
-        newComponents
-            .putPcollections(feedSDFCollectionId, feedSDFCollection)
-            .putTransforms(feedSDFId, feedSDF);
-        newPTransform.addSubtransforms(feedSDFId);
-      }
-
-      // === Construct the SPLITTABLE_PROCESS_ELEMENTS operation
-      {
-        String runSDFId =
-            uniqueId(String.format("%s/RunSDF", spkId), components::containsTransforms);
-        PTransform runSDF =
-            PTransform.newBuilder()
-                .setUniqueName(String.format("%s/RunSDF", spk.getUniqueName()))
-                .putInputs("input", feedSDFCollectionId)
-                .setSpec(
-                    FunctionSpec.newBuilder()
-                        .setUrn(PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN)
-                        .setPayload(spk.getSpec().getPayload()))
-                .putAllOutputs(spk.getOutputsMap())
-                .build();
-        newComponents.putTransforms(runSDFId, runSDF);
-        newPTransform.addSubtransforms(runSDFId);
-      }
-
-      return MessageWithComponents.newBuilder()
-          .setPtransform(newPTransform.build())
-          .setComponents(newComponents)
-          .build();
-    }
-  }
-
-  /**
-   * Finds FEED_SDF nodes followed by an ExecutableStage and replaces them by a single {@link
-   * SplittableRemoteStageEvaluatorFactory#URN} stage that feeds the ExecutableStage knowing that
-   * the first instruction in the stage is an SDF.
-   */
-  private static Pipeline foldFeedSDFIntoExecutableStage(Pipeline p) {
-    Pipeline.Builder newPipeline = p.toBuilder();
-    Components.Builder newPipelineComponents = newPipeline.getComponentsBuilder();
-
-    QueryablePipeline q = QueryablePipeline.forPipeline(p);
-    String feedSdfUrn = SplittableRemoteStageEvaluatorFactory.FEED_SDF_URN;
-    List<PTransformNode> feedSDFNodes =
-        q.getTransforms().stream()
-            .filter(node -> node.getTransform().getSpec().getUrn().equals(feedSdfUrn))
-            .collect(Collectors.toList());
-    Map<String, PTransformNode> stageToFeeder = Maps.newHashMap();
-    for (PTransformNode node : feedSDFNodes) {
-      PCollectionNode output = Iterables.getOnlyElement(q.getOutputPCollections(node));
-      PTransformNode consumer = Iterables.getOnlyElement(q.getPerElementConsumers(output));
-      String consumerUrn = consumer.getTransform().getSpec().getUrn();
-      checkState(
-          consumerUrn.equals(ExecutableStage.URN),
-          "Expected all FeedSDF nodes to be consumed by an ExecutableStage, "
-              + "but %s is consumed by %s which is %s",
-          node.getId(),
-          consumer.getId(),
-          consumerUrn);
-      stageToFeeder.put(consumer.getId(), node);
-    }
-
-    // Copy over root transforms except for the excluded FEED_SDF transforms.
-    Set<String> feedSDFIds =
-        feedSDFNodes.stream().map(PTransformNode::getId).collect(Collectors.toSet());
-    newPipeline.clearRootTransformIds();
-    for (String rootId : p.getRootTransformIdsList()) {
-      if (!feedSDFIds.contains(rootId)) {
-        newPipeline.addRootTransformIds(rootId);
-      }
-    }
-    // Copy over all transforms, except FEED_SDF transforms are skipped, and ExecutableStage's
-    // feeding from them are replaced.
-    for (PTransformNode node : q.getTransforms()) {
-      if (feedSDFNodes.contains(node)) {
-        // These transforms are skipped and handled separately.
-        continue;
-      }
-      if (!stageToFeeder.containsKey(node.getId())) {
-        // This transform is unchanged
-        newPipelineComponents.putTransforms(node.getId(), node.getTransform());
-        continue;
-      }
-      // "node" is an ExecutableStage transform feeding from an SDF.
-      PTransformNode feedSDFNode = stageToFeeder.get(node.getId());
-      PCollectionNode rawGBKOutput =
-          Iterables.getOnlyElement(q.getPerElementInputPCollections(feedSDFNode));
-
-      // Replace the ExecutableStage transform.
-      newPipelineComponents.putTransforms(
-          node.getId(),
-          node.getTransform()
-              .toBuilder()
-              .mergeSpec(
-                  // Change URN from ExecutableStage.URN to URN of the ULR's splittable executable
-                  // stage evaluator.
-                  FunctionSpec.newBuilder()
-                      .setUrn(SplittableRemoteStageEvaluatorFactory.URN)
-                      .build())
-              .putInputs(
-                  // The splittable executable stage now reads from the raw GBK, instead of
-                  // from the now non-existent FEED_SDF.
-                  Iterables.getOnlyElement(node.getTransform().getInputsMap().keySet()),
-                  rawGBKOutput.getId())
-              .build());
-    }
-
-    return newPipeline.build();
-  }
-
-  private enum EnvironmentType {
-    DOCKER,
-    IN_PROCESS
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RemoteStageEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RemoteStageEvaluatorFactory.java
deleted file mode 100644
index 335c1e7..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RemoteStageEvaluatorFactory.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.fnexecution.control.BundleProgressHandler;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.runners.fnexecution.control.RemoteBundle;
-import org.apache.beam.runners.fnexecution.control.StageBundleFactory;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-
-/**
- * The {@link TransformEvaluatorFactory} which produces {@link TransformEvaluator evaluators} for
- * stages which execute on an SDK harness via the Fn Execution APIs.
- */
-class RemoteStageEvaluatorFactory implements TransformEvaluatorFactory {
-  private final BundleFactory bundleFactory;
-
-  private final JobBundleFactory jobFactory;
-
-  RemoteStageEvaluatorFactory(BundleFactory bundleFactory, JobBundleFactory jobFactory) {
-    this.bundleFactory = bundleFactory;
-    this.jobFactory = jobFactory;
-  }
-
-  @Nullable
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) throws Exception {
-    return new RemoteStageEvaluator<>(application);
-  }
-
-  @Override
-  public void cleanup() throws Exception {
-    jobFactory.close();
-  }
-
-  private class RemoteStageEvaluator<T> implements TransformEvaluator<T> {
-    private final PTransformNode transform;
-    private final RemoteBundle bundle;
-    private final FnDataReceiver<WindowedValue<?>> mainInput;
-    private final Collection<UncommittedBundle<?>> outputs;
-
-    private RemoteStageEvaluator(PTransformNode transform) throws Exception {
-      this.transform = transform;
-      ExecutableStage stage =
-          ExecutableStage.fromPayload(
-              ExecutableStagePayload.parseFrom(transform.getTransform().getSpec().getPayload()));
-      this.outputs = new ArrayList<>();
-      StageBundleFactory stageFactory = jobFactory.forStage(stage);
-      this.bundle =
-          stageFactory.getBundle(
-              BundleFactoryOutputReceiverFactory.create(
-                  bundleFactory, stage.getComponents(), outputs::add),
-              StateRequestHandler.unsupported(),
-              BundleProgressHandler.ignored());
-      // TODO(BEAM-4680): Add support for timers as inputs to the ULR
-      this.mainInput = Iterables.getOnlyElement(bundle.getInputReceivers().values());
-    }
-
-    @Override
-    public void processElement(WindowedValue<T> element) throws Exception {
-      mainInput.accept(element);
-    }
-
-    @Override
-    public TransformResult<T> finishBundle() throws Exception {
-      bundle.close();
-      return StepTransformResult.<T>withoutHold(transform).addOutput(outputs).build();
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootInputProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootInputProvider.java
deleted file mode 100644
index 9f904c7..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootInputProvider.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-
-/**
- * Provides {@link CommittedBundle bundles} that will be provided to the {@link PTransform
- * PTransforms} that are at the root of a {@link Pipeline}.
- */
-interface RootInputProvider<ShardT> {
-  /**
-   * Get the initial inputs for the {@link PTransformNode}. The {@link PTransformNode} will be
-   * provided with these {@link CommittedBundle bundles} as input when the {@link Pipeline} runs.
-   *
-   * <p>For source transforms, these should be sufficient that, when provided to the evaluators
-   * produced by {@link TransformEvaluatorFactory#forApplication(PTransformNode, CommittedBundle)},
-   * all of the elements contained in the source are eventually produced.
-   *
-   * @param transform the {@link PTransformNode} to get initial inputs for.
-   * @param targetParallelism the target amount of parallelism to obtain from the source. Must be
-   *     greater than or equal to 1.
-   */
-  Collection<CommittedBundle<ShardT>> getInitialInputs(
-      PTransformNode transform, int targetParallelism) throws Exception;
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootProviderRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootProviderRegistry.java
deleted file mode 100644
index 054a04d..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootProviderRegistry.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN;
-import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.Map;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
-
-/**
- * A {@link RootInputProvider} that delegates to primitive {@link RootInputProvider} implementations
- * based on the type of {@link PTransform} of the application.
- */
-class RootProviderRegistry {
-
-  /**
-   * Returns a {@link RootProviderRegistry} that supports the {@link Impulse} and {@link Flatten}
-   * primitives.
-   */
-  static RootProviderRegistry javaPortableRegistry(BundleFactory bundleFactory) {
-    return new RootProviderRegistry(
-        ImmutableMap.<String, RootInputProvider<?>>builder()
-            .put(
-                IMPULSE_TRANSFORM_URN,
-                new ImpulseEvaluatorFactory.ImpulseRootProvider(bundleFactory))
-            .put(FLATTEN_TRANSFORM_URN, new EmptyInputProvider())
-            .build());
-  }
-
-  private final Map<String, RootInputProvider<?>> providers;
-
-  private RootProviderRegistry(Map<String, RootInputProvider<?>> providers) {
-    this.providers = providers;
-  }
-
-  public Collection<CommittedBundle<?>> getInitialInputs(
-      PTransformNode transform, int targetParallelism) throws Exception {
-    String transformUrn = PTransformTranslation.urnForTransformOrNull(transform.getTransform());
-    RootInputProvider provider =
-        checkNotNull(
-            providers.get(transformUrn),
-            "Tried to get a %s for a transform \"%s\", but there is no such provider",
-            RootInputProvider.class.getSimpleName(),
-            transformUrn);
-    return provider.getInitialInputs(transform, targetParallelism);
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SourceShard.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SourceShard.java
deleted file mode 100644
index cf978b8..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SourceShard.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.Source;
-import org.apache.beam.sdk.io.UnboundedSource;
-
-/**
- * A shard for a source in the {@link Read} transform.
- *
- * <p>Since {@link UnboundedSource} and {@link BoundedSource} have radically different needs, this
- * is a mostly-empty interface.
- */
-interface SourceShard<T> {
-  Source<T> getSource();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SplittableRemoteStageEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SplittableRemoteStageEvaluatorFactory.java
deleted file mode 100644
index 28f8789..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SplittableRemoteStageEvaluatorFactory.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleProgressResponse;
-import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.fnexecution.control.BundleProgressHandler;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.runners.fnexecution.control.RemoteBundle;
-import org.apache.beam.runners.fnexecution.splittabledofn.SDFFeederViaStateAndTimers;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-
-/**
- * The {@link TransformEvaluatorFactory} for {@link #URN}, which reads from a {@link
- * DirectGroupByKey#DIRECT_GBKO_URN} and feeds the data, using state and timers, to a {@link
- * ExecutableStage} whose first instruction is an SDF.
- */
-class SplittableRemoteStageEvaluatorFactory implements TransformEvaluatorFactory {
-  public static final String URN = "beam:directrunner:transforms:splittable_remote_stage:v1";
-
-  // A fictional transform that transforms from KWI<unique key, KV<element, restriction>>
-  // to simply KV<element, restriction> taken by the SDF inside the ExecutableStage.
-  public static final String FEED_SDF_URN = "beam:directrunner:transforms:feed_sdf:v1";
-
-  private final BundleFactory bundleFactory;
-  private final JobBundleFactory jobBundleFactory;
-  private final StepStateAndTimers.Provider stp;
-
-  SplittableRemoteStageEvaluatorFactory(
-      BundleFactory bundleFactory,
-      JobBundleFactory jobBundleFactory,
-      StepStateAndTimers.Provider stepStateAndTimers) {
-    this.bundleFactory = bundleFactory;
-    this.jobBundleFactory = jobBundleFactory;
-    this.stp = stepStateAndTimers;
-  }
-
-  @Nullable
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) throws Exception {
-    return new SplittableRemoteStageEvaluator(
-        bundleFactory,
-        jobBundleFactory,
-        stp.forStepAndKey(application, inputBundle.getKey()),
-        application);
-  }
-
-  @Override
-  public void cleanup() throws Exception {
-    jobBundleFactory.close();
-  }
-
-  private static class SplittableRemoteStageEvaluator<InputT, RestrictionT>
-      implements TransformEvaluator<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> {
-    private final PTransformNode transform;
-    private final ExecutableStage stage;
-
-    private final CopyOnAccessInMemoryStateInternals<byte[]> stateInternals;
-    private final DirectTimerInternals timerInternals;
-    private final RemoteBundle bundle;
-    private final FnDataReceiver<WindowedValue<?>> mainInput;
-    private final Collection<UncommittedBundle<?>> outputs;
-
-    private final SDFFeederViaStateAndTimers<InputT, RestrictionT> feeder;
-
-    private SplittableRemoteStageEvaluator(
-        BundleFactory bundleFactory,
-        JobBundleFactory jobBundleFactory,
-        StepStateAndTimers<byte[]> stp,
-        PTransformNode transform)
-        throws Exception {
-      this.stateInternals = stp.stateInternals();
-      this.timerInternals = stp.timerInternals();
-      this.transform = transform;
-      this.stage =
-          ExecutableStage.fromPayload(
-              ExecutableStagePayload.parseFrom(transform.getTransform().getSpec().getPayload()));
-      this.outputs = new ArrayList<>();
-
-      FullWindowedValueCoder<KV<InputT, RestrictionT>> windowedValueCoder =
-          (FullWindowedValueCoder<KV<InputT, RestrictionT>>)
-              WireCoders.<KV<InputT, RestrictionT>>instantiateRunnerWireCoder(
-                  stage.getInputPCollection(), stage.getComponents());
-      KvCoder<InputT, RestrictionT> kvCoder =
-          (KvCoder<InputT, RestrictionT>) windowedValueCoder.getValueCoder();
-      this.feeder =
-          new SDFFeederViaStateAndTimers<>(
-              stateInternals,
-              timerInternals,
-              kvCoder.getKeyCoder(),
-              kvCoder.getValueCoder(),
-              (Coder<BoundedWindow>) windowedValueCoder.getWindowCoder());
-
-      this.bundle =
-          jobBundleFactory
-              .forStage(stage)
-              .getBundle(
-                  BundleFactoryOutputReceiverFactory.create(
-                      bundleFactory, stage.getComponents(), outputs::add),
-                  StateRequestHandler.unsupported(),
-                  // TODO: Wire in splitting via a split listener
-                  new BundleProgressHandler() {
-                    @Override
-                    public void onProgress(ProcessBundleProgressResponse progress) {}
-
-                    @Override
-                    public void onCompleted(ProcessBundleResponse response) {}
-                  });
-      this.mainInput = Iterables.getOnlyElement(bundle.getInputReceivers().values());
-    }
-
-    @Override
-    public void processElement(
-        WindowedValue<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> windowedWorkItem)
-        throws Exception {
-      KeyedWorkItem<byte[], KV<InputT, RestrictionT>> kwi = windowedWorkItem.getValue();
-      WindowedValue<KV<InputT, RestrictionT>> elementRestriction =
-          Iterables.getOnlyElement(kwi.elementsIterable(), null);
-      if (elementRestriction != null) {
-        feeder.seed(elementRestriction);
-      } else {
-        elementRestriction = feeder.resume(Iterables.getOnlyElement(kwi.timersIterable()));
-      }
-      mainInput.accept(elementRestriction);
-    }
-
-    @Override
-    public TransformResult<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> finishBundle()
-        throws Exception {
-      bundle.close();
-      feeder.commit();
-      CopyOnAccessInMemoryStateInternals<byte[]> state = stateInternals.commit();
-      StepTransformResult.Builder<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> result =
-          StepTransformResult.withHold(transform, state.getEarliestWatermarkHold());
-      return result
-          .addOutput(outputs)
-          .withState(state)
-          .withTimerUpdate(timerInternals.getTimerUpdate())
-          .build();
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepAndKey.java
deleted file mode 100644
index 4c4a48b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepAndKey.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Objects;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.MoreObjects;
-
-/**
- * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
- * per-step in a keyed manner (e.g. State).
- */
-final class StepAndKey {
-  private final PTransformNode step;
-  private final StructuralKey<?> key;
-
-  /** Create a new {@link StepAndKey} with the provided step and key. */
-  public static StepAndKey of(PTransformNode step, StructuralKey<?> key) {
-    return new StepAndKey(step, key);
-  }
-
-  private StepAndKey(PTransformNode step, StructuralKey<?> key) {
-    this.step = step;
-    this.key = key;
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(StepAndKey.class)
-        .add("step", step.getId())
-        .add("key", key.getKey())
-        .toString();
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(step, key);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == this) {
-      return true;
-    } else if (!(other instanceof StepAndKey)) {
-      return false;
-    } else {
-      StepAndKey that = (StepAndKey) other;
-      return Objects.equals(this.step, that.step) && Objects.equals(this.key, that.key);
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepStateAndTimers.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepStateAndTimers.java
deleted file mode 100644
index e5cfe55..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepStateAndTimers.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.local.StructuralKey;
-
-/** A provider of {@link StateInternals} and {@link TimerInternals}. */
-interface StepStateAndTimers<K> {
-  interface Provider {
-    <K> StepStateAndTimers<K> forStepAndKey(PTransformNode transform, StructuralKey<K> key);
-  }
-
-  CopyOnAccessInMemoryStateInternals<K> stateInternals();
-
-  DirectTimerInternals timerInternals();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepTransformResult.java
deleted file mode 100644
index b694d3b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepTransformResult.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import com.google.auto.value.AutoValue;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.joda.time.Instant;
-
-/** An immutable {@link TransformResult}. */
-@AutoValue
-abstract class StepTransformResult<InputT> implements TransformResult<InputT> {
-
-  public static <InputT> Builder<InputT> withHold(PTransformNode transform, Instant watermarkHold) {
-    return new Builder(transform, watermarkHold);
-  }
-
-  public static <InputT> Builder<InputT> withoutHold(PTransformNode transform) {
-    return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
-  }
-
-  @Override
-  public TransformResult<InputT> withLogicalMetricUpdates(MetricUpdates metricUpdates) {
-    return new AutoValue_StepTransformResult(
-        getTransform(),
-        getOutputBundles(),
-        getUnprocessedElements(),
-        metricUpdates,
-        getWatermarkHold(),
-        getState(),
-        getTimerUpdate(),
-        getOutputTypes());
-  }
-
-  /** A builder for creating instances of {@link StepTransformResult}. */
-  public static class Builder<InputT> {
-    private final PTransformNode transform;
-    private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
-    private final ImmutableList.Builder<WindowedValue<InputT>> unprocessedElementsBuilder;
-    private MetricUpdates metricUpdates;
-    private CopyOnAccessInMemoryStateInternals state;
-    private TimerUpdate timerUpdate;
-    private final Set<OutputType> producedOutputs;
-    private final Instant watermarkHold;
-
-    private Builder(PTransformNode transform, Instant watermarkHold) {
-      this.transform = transform;
-      this.watermarkHold = watermarkHold;
-      this.bundlesBuilder = ImmutableList.builder();
-      this.producedOutputs = EnumSet.noneOf(OutputType.class);
-      this.unprocessedElementsBuilder = ImmutableList.builder();
-      this.timerUpdate = TimerUpdate.builder(null).build();
-      this.metricUpdates = MetricUpdates.EMPTY;
-    }
-
-    public StepTransformResult<InputT> build() {
-      return new AutoValue_StepTransformResult<>(
-          transform,
-          bundlesBuilder.build(),
-          unprocessedElementsBuilder.build(),
-          metricUpdates,
-          watermarkHold,
-          state,
-          timerUpdate,
-          producedOutputs);
-    }
-
-    public Builder<InputT> withMetricUpdates(MetricUpdates metricUpdates) {
-      this.metricUpdates = metricUpdates;
-      return this;
-    }
-
-    public Builder<InputT> withState(CopyOnAccessInMemoryStateInternals state) {
-      this.state = state;
-      return this;
-    }
-
-    public Builder<InputT> withTimerUpdate(TimerUpdate timerUpdate) {
-      this.timerUpdate = timerUpdate;
-      return this;
-    }
-
-    public Builder<InputT> addUnprocessedElements(WindowedValue<InputT>... unprocessed) {
-      unprocessedElementsBuilder.addAll(Arrays.asList(unprocessed));
-      return this;
-    }
-
-    public Builder<InputT> addUnprocessedElements(
-        Iterable<? extends WindowedValue<InputT>> unprocessed) {
-      unprocessedElementsBuilder.addAll(unprocessed);
-      return this;
-    }
-
-    public Builder<InputT> addOutput(
-        UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
-      bundlesBuilder.add(outputBundle);
-      bundlesBuilder.add(outputBundles);
-      return this;
-    }
-
-    public Builder<InputT> addOutput(Collection<UncommittedBundle<?>> outputBundles) {
-      bundlesBuilder.addAll(outputBundles);
-      return this;
-    }
-
-    public Builder<InputT> withAdditionalOutput(OutputType producedAdditionalOutput) {
-      producedOutputs.add(producedAdditionalOutput);
-      return this;
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluator.java
deleted file mode 100644
index 2579bcc..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluator.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An evaluator of a specific application of a transform. Will be used for at least one {@link
- * CommittedBundle}.
- *
- * @param <InputT> the type of elements that will be passed to {@link #processElement}
- */
-interface TransformEvaluator<InputT> {
-  /**
-   * Process an element in the input {@link CommittedBundle}.
-   *
-   * @param element the element to process
-   */
-  void processElement(WindowedValue<InputT> element) throws Exception;
-
-  /**
-   * Finish processing the bundle of this {@link TransformEvaluator}.
-   *
-   * <p>After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
-   * and no more elements will be processed.
-   *
-   * @return an {@link TransformResult} containing the results of this bundle evaluation.
-   */
-  TransformResult<InputT> finishBundle() throws Exception;
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorFactory.java
deleted file mode 100644
index 9892c36..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorFactory.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.transforms.PTransform;
-
-/**
- * A factory for creating instances of {@link TransformEvaluator} for the application of a {@link
- * PTransform}.
- *
- * <p>{@link TransformEvaluatorFactory TransformEvaluatorFactories} will be reused within a single
- * execution of a {@link Pipeline} but will not be reused across executions.
- */
-interface TransformEvaluatorFactory {
-  /**
-   * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
-   *
-   * <p>Any work that must be done before input elements are processed (such as calling {@code
-   * DoFn.StartBundle}) must be done before the {@link TransformEvaluator} is made available to the
-   * caller.
-   *
-   * <p>May return null if the application cannot produce an evaluator (for example, it is a {@link
-   * Read} {@link PTransform} where all evaluators are in-use).
-   *
-   * @return An evaluator capable of processing the transform on the bundle, or null if no evaluator
-   *     can be constructed.
-   * @throws Exception whenever constructing the underlying evaluator throws an exception
-   */
-  @Nullable
-  <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) throws Exception;
-
-  /**
-   * Cleans up any state maintained by this {@link TransformEvaluatorFactory}. Called after a {@link
-   * Pipeline} is shut down. No more calls to {@link #forApplication(PTransformNode,
-   * CommittedBundle)} will be made after a call to {@link #cleanup()}.
-   */
-  void cleanup() throws Exception;
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorRegistry.java
deleted file mode 100644
index 6587882..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorRegistry.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
- * implementations based on the type of {@link PTransform} of the application.
- */
-class TransformEvaluatorRegistry {
-  private static final Logger LOG = LoggerFactory.getLogger(TransformEvaluatorRegistry.class);
-
-  static TransformEvaluatorRegistry portableRegistry(
-      ExecutableGraph<PTransformNode, PCollectionNode> graph,
-      Components components,
-      BundleFactory bundleFactory,
-      JobBundleFactory jobBundleFactory,
-      StepStateAndTimers.Provider stepStateAndTimers) {
-    return new TransformEvaluatorRegistry(
-        ImmutableMap.<String, TransformEvaluatorFactory>builder()
-            .put(
-                PTransformTranslation.IMPULSE_TRANSFORM_URN,
-                new ImpulseEvaluatorFactory(graph, bundleFactory))
-            .put(
-                PTransformTranslation.FLATTEN_TRANSFORM_URN,
-                new FlattenEvaluatorFactory(graph, bundleFactory))
-            .put(
-                DirectGroupByKey.DIRECT_GBKO_URN,
-                new GroupByKeyOnlyEvaluatorFactory(graph, components, bundleFactory))
-            .put(
-                DirectGroupByKey.DIRECT_GABW_URN,
-                new GroupAlsoByWindowEvaluatorFactory(
-                    graph, components, bundleFactory, stepStateAndTimers))
-            .put(
-                ExecutableStage.URN,
-                new RemoteStageEvaluatorFactory(bundleFactory, jobBundleFactory))
-            .put(
-                SplittableRemoteStageEvaluatorFactory.URN,
-                new SplittableRemoteStageEvaluatorFactory(
-                    bundleFactory, jobBundleFactory, stepStateAndTimers))
-            .build());
-  }
-
-  // the TransformEvaluatorFactories can construct instances of all generic types of transform,
-  // so all instances of a primitive can be handled with the same evaluator factory.
-  private final Map<String, TransformEvaluatorFactory> factories;
-
-  private final AtomicBoolean finished = new AtomicBoolean(false);
-
-  private TransformEvaluatorRegistry(
-      @SuppressWarnings("rawtypes") Map<String, TransformEvaluatorFactory> factories) {
-    this.factories = factories;
-  }
-
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, CommittedBundle<?> inputBundle) throws Exception {
-    checkState(
-        !finished.get(), "Tried to get an evaluator for a finished TransformEvaluatorRegistry");
-
-    String urn = PTransformTranslation.urnForTransformOrNull(application.getTransform());
-
-    TransformEvaluatorFactory factory =
-        checkNotNull(factories.get(urn), "No evaluator for PTransform \"%s\"", urn);
-    return factory.forApplication(application, inputBundle);
-  }
-
-  public void cleanup() throws Exception {
-    Collection<Exception> thrownInCleanup = new ArrayList<>();
-    for (TransformEvaluatorFactory factory : factories.values()) {
-      try {
-        factory.cleanup();
-      } catch (Exception e) {
-        if (e instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        thrownInCleanup.add(e);
-      }
-    }
-    finished.set(true);
-    if (!thrownInCleanup.isEmpty()) {
-      LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup);
-      Exception toThrow = null;
-      for (Exception e : thrownInCleanup) {
-        if (toThrow == null) {
-          toThrow = e;
-        } else {
-          toThrow.addSuppressed(e);
-        }
-      }
-      throw toThrow;
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutor.java
deleted file mode 100644
index f4f34cc..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutor.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-/** A {@link Runnable} that will execute a {@code PTransform} on some bundle of input. */
-public interface TransformExecutor extends Runnable {}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorFactory.java
deleted file mode 100644
index fe29aee..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorFactory.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-
-/** A Factory for creating {@link TransformExecutor Transform Executors} on an input. */
-interface TransformExecutorFactory {
-  TransformExecutor create(
-      CommittedBundle<?> bundle,
-      PTransformNode transform,
-      CompletionCallback onComplete,
-      TransformExecutorService executorService);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorService.java
deleted file mode 100644
index f8dcf8d..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorService.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-/**
- * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
- * appropriate for the {@link StepAndKey} the executor exists for.
- */
-interface TransformExecutorService {
-  /** Schedule the provided work to be eventually executed. */
-  void schedule(TransformExecutor work);
-
-  /**
-   * Finish executing the provided work. This may cause additional {@link DirectTransformExecutor
-   * TransformExecutors} to be evaluated.
-   */
-  void complete(TransformExecutor completed);
-
-  /**
-   * Cancel any outstanding work, if possible. Any future calls to schedule should ignore any work.
-   */
-  void shutdown();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorServices.java
deleted file mode 100644
index 24d7694..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorServices.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.MoreObjects;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Static factory methods for constructing instances of {@link TransformExecutorService}. */
-final class TransformExecutorServices {
-  private TransformExecutorServices() {
-    // Do not instantiate
-  }
-
-  /**
-   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
-   * parallel.
-   */
-  public static TransformExecutorService parallel(ExecutorService executor) {
-    return new ParallelTransformExecutor(executor);
-  }
-
-  /**
-   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
-   * serial.
-   */
-  public static TransformExecutorService serial(ExecutorService executor) {
-    return new SerialTransformExecutor(executor);
-  }
-
-  /**
-   * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
-   * scheduled will be immediately submitted to the {@link ExecutorService}.
-   *
-   * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
-   * processed in parallel.
-   */
-  private static class ParallelTransformExecutor implements TransformExecutorService {
-    private static final Logger LOG = LoggerFactory.getLogger(ParallelTransformExecutor.class);
-
-    private final ExecutorService executor;
-    private final AtomicBoolean active = new AtomicBoolean(true);
-
-    private ParallelTransformExecutor(ExecutorService executor) {
-      this.executor = executor;
-    }
-
-    @Override
-    // TODO: [BEAM-4563] Pass Future back to consumer to check for async errors
-    @SuppressWarnings("FutureReturnValueIgnored")
-    public void schedule(TransformExecutor work) {
-      if (active.get()) {
-        try {
-          executor.submit(work);
-        } catch (RejectedExecutionException rejected) {
-          boolean stillActive = active.get();
-          if (stillActive) {
-            throw new IllegalStateException(
-                String.format(
-                    "Execution of Work %s was rejected, but the %s is still active",
-                    work, ParallelTransformExecutor.class.getSimpleName()));
-          } else {
-            LOG.debug(
-                "Rejected execution of Work {} on executor {}. "
-                    + "Suppressed exception because evaluator is not active",
-                work,
-                this);
-          }
-        }
-      }
-    }
-
-    @Override
-    public void complete(TransformExecutor completed) {}
-
-    @Override
-    public void shutdown() {
-      active.set(false);
-    }
-  }
-
-  /**
-   * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
-   * scheduled will be placed on the work queue. Only one item of work will be submitted to the
-   * {@link ExecutorService} at any time.
-   *
-   * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair. Keyed
-   * computations are processed serially per step.
-   */
-  private static class SerialTransformExecutor implements TransformExecutorService {
-    private final ExecutorService executor;
-
-    private AtomicReference<TransformExecutor> currentlyEvaluating;
-    private final Queue<TransformExecutor> workQueue;
-    private boolean active = true;
-
-    private SerialTransformExecutor(ExecutorService executor) {
-      this.executor = executor;
-      this.currentlyEvaluating = new AtomicReference<>();
-      this.workQueue = new ConcurrentLinkedQueue<>();
-    }
-
-    /**
-     * Schedules the work, adding it to the work queue if there is a bundle currently being
-     * evaluated and scheduling it immediately otherwise.
-     */
-    @Override
-    public void schedule(TransformExecutor work) {
-      workQueue.offer(work);
-      updateCurrentlyEvaluating();
-    }
-
-    @Override
-    public void complete(TransformExecutor completed) {
-      if (!currentlyEvaluating.compareAndSet(completed, null)) {
-        throw new IllegalStateException(
-            "Finished work "
-                + completed
-                + " but could not complete due to unexpected currently executing "
-                + currentlyEvaluating.get());
-      }
-      updateCurrentlyEvaluating();
-    }
-
-    @Override
-    public void shutdown() {
-      synchronized (this) {
-        active = false;
-      }
-      workQueue.clear();
-    }
-
-    // TODO: [BEAM-4563] Pass Future back to consumer to check for async errors
-    @SuppressWarnings("FutureReturnValueIgnored")
-    private void updateCurrentlyEvaluating() {
-      if (currentlyEvaluating.get() == null) {
-        // Only synchronize if we need to update what's currently evaluating
-        synchronized (this) {
-          TransformExecutor newWork = workQueue.poll();
-          if (active && newWork != null) {
-            if (currentlyEvaluating.compareAndSet(null, newWork)) {
-              executor.submit(newWork);
-            } else {
-              workQueue.offer(newWork);
-            }
-          }
-        }
-      }
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(SerialTransformExecutor.class)
-          .add("currentlyEvaluating", currentlyEvaluating)
-          .add("workQueue", workQueue)
-          .toString();
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformResult.java
deleted file mode 100644
index 3f055e4..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformResult.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.joda.time.Instant;
-
-/**
- * The result of evaluating an {@link PTransformNode} with a {@link TransformEvaluator}.
- *
- * <p>Every transform evaluator has a defined input type, but {@link ParDo} has multiple outputs so
- * there is not necesssarily a defined output type.
- */
-interface TransformResult<InputT> {
-  /**
-   * Returns the {@link PTransformNode} that produced this result.
-   *
-   * <p>This is treated as an opaque identifier so evaluators can delegate to other evaluators that
-   * may not have compatible types.
-   */
-  PTransformNode getTransform();
-
-  /**
-   * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
-   * will be committed by the evaluation context as part of completing this result.
-   *
-   * <p>Note that the bundles need not have a uniform type, for example in the case of multi-output
-   * {@link ParDo}.
-   */
-  Iterable<? extends UncommittedBundle<?>> getOutputBundles();
-
-  /**
-   * Returns elements that were provided to the {@link TransformEvaluator} as input but were not
-   * processed.
-   */
-  Iterable<? extends WindowedValue<InputT>> getUnprocessedElements();
-
-  /** Returns the logical metric updates. */
-  MetricUpdates getLogicalMetricUpdates();
-
-  /**
-   * Returns the Watermark Hold for the transform at the time this result was produced.
-   *
-   * <p>If the transform does not set any watermark hold, returns {@link
-   * BoundedWindow#TIMESTAMP_MAX_VALUE}.
-   */
-  Instant getWatermarkHold();
-
-  /**
-   * Returns the State used by the transform.
-   *
-   * <p>If this evaluation did not access state, this may return null.
-   */
-  @Nullable
-  CopyOnAccessInMemoryStateInternals getState();
-
-  /**
-   * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
-   * evaluation was triggered due to the delivery of one or more timers, those timers must be added
-   * to the builder before it is complete.
-   *
-   * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
-   */
-  TimerUpdate getTimerUpdate();
-
-  /**
-   * Returns the types of output produced by this {@link PTransform}. This may not include {@link
-   * OutputType#BUNDLE}, as empty bundles may be dropped when the transform is committed.
-   */
-  Set<OutputType> getOutputTypes();
-
-  /**
-   * Returns a new TransformResult based on this one but overwriting any existing logical metric
-   * updates with {@code metricUpdates}.
-   */
-  TransformResult<InputT> withLogicalMetricUpdates(MetricUpdates metricUpdates);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/UncommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/UncommittedBundle.java
deleted file mode 100644
index a55d415..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/UncommittedBundle.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-
-/**
- * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
- * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is a
- * part of at a later point. This is an uncommitted bundle and can have elements added to it.
- *
- * @param <T> the type of elements that can be added to this bundle
- */
-interface UncommittedBundle<T> {
-  /** Returns the PCollection that the elements of this {@link UncommittedBundle} belong to. */
-  @Nullable
-  PCollectionNode getPCollection();
-
-  /**
-   * Outputs an element to this bundle.
-   *
-   * @param element the element to add to this bundle
-   * @return this bundle
-   */
-  UncommittedBundle<T> add(WindowedValue<T> element);
-
-  /**
-   * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
-   * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
-   * will throw an {@link IllegalStateException} if called after a call to commit.
-   *
-   * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
-   *     committed
-   */
-  CommittedBundle<T> commit(Instant synchronizedProcessingTime);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WatermarkCallbackExecutor.java
deleted file mode 100644
index b7c8a83..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WatermarkCallbackExecutor.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.io.Serializable;
-import java.util.PriorityQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executor;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ComparisonChain;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Ordering;
-import org.joda.time.Instant;
-
-/**
- * Executes callbacks that occur based on the progression of the watermark per-step.
- *
- * <p>Callbacks are registered by calls to {@link #callOnGuaranteedFiring(PTransformNode,
- * BoundedWindow, WindowingStrategy, Runnable)}, and are executed after a call to {@link
- * #fireForWatermark(PTransformNode, Instant)} with the same {@link PTransformNode} and a watermark
- * sufficient to ensure that the trigger for the windowing strategy would have been produced.
- *
- * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
- * {@link PTransformNode} - any call to {@link #callOnGuaranteedFiring(PTransformNode,
- * BoundedWindow, WindowingStrategy, Runnable)} that could have potentially already fired should be
- * followed by a call to {@link #fireForWatermark(PTransformNode, Instant)} for the same transform
- * with the current value of the watermark.
- */
-class WatermarkCallbackExecutor {
-  /** Create a new {@link WatermarkCallbackExecutor}. */
-  public static WatermarkCallbackExecutor create(Executor executor) {
-    return new WatermarkCallbackExecutor(executor);
-  }
-
-  private final ConcurrentMap<PTransformNode, PriorityQueue<WatermarkCallback>> callbacks;
-  private final Executor executor;
-
-  private WatermarkCallbackExecutor(Executor executor) {
-    this.callbacks = new ConcurrentHashMap<>();
-    this.executor = executor;
-  }
-
-  /**
-   * Execute the provided {@link Runnable} after the next call to {@link
-   * #fireForWatermark(PTransformNode, Instant)} where the window is guaranteed to have produced
-   * output.
-   */
-  public void callOnGuaranteedFiring(
-      PTransformNode step,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    WatermarkCallback callback =
-        WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
-
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
-      if (callbacks.putIfAbsent(step, callbackQueue) != null) {
-        callbackQueue = callbacks.get(step);
-      }
-    }
-
-    synchronized (callbackQueue) {
-      callbackQueue.offer(callback);
-    }
-  }
-
-  /**
-   * Execute the provided {@link Runnable} after the next call to {@link
-   * #fireForWatermark(PTransformNode, Instant)} where the window is guaranteed to be expired.
-   */
-  public void callOnWindowExpiration(
-      PTransformNode step,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    WatermarkCallback callback =
-        WatermarkCallback.afterWindowExpiration(window, windowingStrategy, runnable);
-
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
-      if (callbacks.putIfAbsent(step, callbackQueue) != null) {
-        callbackQueue = callbacks.get(step);
-      }
-    }
-
-    synchronized (callbackQueue) {
-      callbackQueue.offer(callback);
-    }
-  }
-
-  /**
-   * Schedule all pending callbacks that must have produced output by the time of the provided
-   * watermark.
-   */
-  public void fireForWatermark(PTransformNode step, Instant watermark) {
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      return;
-    }
-    synchronized (callbackQueue) {
-      while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
-        executor.execute(callbackQueue.poll().getCallback());
-      }
-    }
-  }
-
-  private static class WatermarkCallback {
-    public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
-        BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
-      @SuppressWarnings("unchecked")
-      Instant firingAfter = strategy.getTrigger().getWatermarkThatGuaranteesFiring((W) window);
-      return new WatermarkCallback(firingAfter, callback);
-    }
-
-    public static <W extends BoundedWindow> WatermarkCallback afterWindowExpiration(
-        BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
-      // Fire one milli past the end of the window. This ensures that all window expiration
-      // timers are delivered first
-      Instant firingAfter = window.maxTimestamp().plus(strategy.getAllowedLateness()).plus(1L);
-      return new WatermarkCallback(firingAfter, callback);
-    }
-
-    private final Instant fireAfter;
-    private final Runnable callback;
-
-    private WatermarkCallback(Instant fireAfter, Runnable callback) {
-      this.fireAfter = fireAfter;
-      this.callback = callback;
-    }
-
-    public boolean shouldFire(Instant currentWatermark) {
-      return currentWatermark.isAfter(fireAfter)
-          || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    }
-
-    public Runnable getCallback() {
-      return callback;
-    }
-  }
-
-  private static class CallbackOrdering extends Ordering<WatermarkCallback>
-      implements Serializable {
-    @Override
-    public int compare(WatermarkCallback left, WatermarkCallback right) {
-      return ComparisonChain.start()
-          .compare(left.fireAfter, right.fireAfter)
-          .compare(left.callback, right.callback, Ordering.arbitrary())
-          .result();
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WindowEvaluatorFactory.java
deleted file mode 100644
index 83d745d..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WindowEvaluatorFactory.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.joda.time.Instant;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link Window.Assign}
- * primitive {@link PTransform}.
- */
-class WindowEvaluatorFactory implements TransformEvaluatorFactory {
-  private final EvaluationContext evaluationContext;
-
-  WindowEvaluatorFactory(EvaluationContext evaluationContext) {
-    this.evaluationContext = evaluationContext;
-  }
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      PTransformNode application, @Nullable CommittedBundle<?> inputBundle) {
-    return createTransformEvaluator(application);
-  }
-
-  private <InputT> TransformEvaluator<InputT> createTransformEvaluator(PTransformNode transform) {
-    WindowFn<? super InputT, ?> fn = null;
-
-    PCollectionNode outputPCollection = null;
-    evaluationContext.createBundle(outputPCollection);
-    throw new UnsupportedOperationException("Not yet migrated");
-  }
-
-  @Override
-  public void cleanup() {}
-
-  private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
-    private final PTransformNode transform;
-    private final WindowFn<InputT, ?> windowFn;
-    private final UncommittedBundle<InputT> outputBundle;
-
-    @SuppressWarnings("unchecked")
-    public WindowIntoEvaluator(
-        PTransformNode transform,
-        WindowFn<? super InputT, ?> windowFn,
-        UncommittedBundle<InputT> outputBundle) {
-      this.outputBundle = outputBundle;
-      this.transform = transform;
-      // Safe contravariant cast
-      this.windowFn = (WindowFn<InputT, ?>) windowFn;
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> compressedElement) throws Exception {
-      for (WindowedValue<InputT> element : compressedElement.explodeWindows()) {
-        Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
-        outputBundle.add(
-            WindowedValue.of(
-                element.getValue(), element.getTimestamp(), windows, element.getPane()));
-      }
-    }
-
-    private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
-        WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
-      WindowFn<InputT, W>.AssignContext assignContext =
-          new DirectAssignContext<>(windowFn, element);
-      return windowFn.assignWindows(assignContext);
-    }
-
-    @Override
-    public TransformResult<InputT> finishBundle() throws Exception {
-      return StepTransformResult.<InputT>withoutHold(transform).addOutput(outputBundle).build();
-    }
-  }
-
-  private static class DirectAssignContext<InputT, W extends BoundedWindow>
-      extends WindowFn<InputT, W>.AssignContext {
-    private final WindowedValue<InputT> value;
-
-    public DirectAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
-      fn.super();
-      this.value = value;
-    }
-
-    @Override
-    public InputT element() {
-      return value.getValue();
-    }
-
-    @Override
-    public Instant timestamp() {
-      return value.getTimestamp();
-    }
-
-    @Override
-    public BoundedWindow window() {
-      return Iterables.getOnlyElement(value.getWindows());
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalArtifactStagingLocation.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalArtifactStagingLocation.java
deleted file mode 100644
index eb9b33b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalArtifactStagingLocation.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.artifact;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.io.File;
-import java.io.IOException;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
-
-/**
- * A location where the results of an {@link LocalFileSystemArtifactStagerService} are stored and
- * where the retrieval service retrieves them from.
- */
-class LocalArtifactStagingLocation {
-  /**
-   * Create a new {@link LocalArtifactStagingLocation} rooted at the specified location, creating
-   * any directories or subdirectories as necessary.
-   */
-  public static LocalArtifactStagingLocation createAt(File rootDirectory) {
-    return new LocalArtifactStagingLocation(rootDirectory).createDirectories();
-  }
-
-  /**
-   * Create a {@link LocalArtifactStagingLocation} for an existing directory. The directory must
-   * contain a manifest and an artifact directory.
-   */
-  public static LocalArtifactStagingLocation forExistingDirectory(File rootDirectory) {
-    return new LocalArtifactStagingLocation(rootDirectory).verifyExistence();
-  }
-
-  private final File rootDirectory;
-  private final File artifactsDirectory;
-
-  private LocalArtifactStagingLocation(File base) {
-    this.rootDirectory = base;
-    this.artifactsDirectory = new File(base, "artifacts");
-  }
-
-  private LocalArtifactStagingLocation createDirectories() {
-    if (((rootDirectory.exists() && rootDirectory.isDirectory()) || rootDirectory.mkdirs())
-        && rootDirectory.canWrite()) {
-      checkState(
-          ((artifactsDirectory.exists() && artifactsDirectory.isDirectory())
-                  || artifactsDirectory.mkdir())
-              && artifactsDirectory.canWrite(),
-          "Could not create artifact staging directory at %s",
-          artifactsDirectory);
-    } else {
-      throw new IllegalStateException(
-          String.format("Could not create staging directory structure at root %s", rootDirectory));
-    }
-    return this;
-  }
-
-  private LocalArtifactStagingLocation verifyExistence() {
-    checkArgument(rootDirectory.exists(), "Nonexistent staging location root %s", rootDirectory);
-    checkArgument(
-        rootDirectory.isDirectory(), "Staging location %s is not a directory", rootDirectory);
-    checkArgument(
-        artifactsDirectory.exists(), "Nonexistent artifact directory %s", artifactsDirectory);
-    checkArgument(
-        artifactsDirectory.isDirectory(),
-        "Artifact location %s is not a directory",
-        artifactsDirectory);
-    checkArgument(getManifestFile().exists(), "No Manifest in existing location %s", rootDirectory);
-    return this;
-  }
-
-  /**
-   * Returns the {@link File} which contains the artifact with the provided name.
-   *
-   * <p>The file may not exist.
-   */
-  public File getArtifactFile(String artifactName) {
-    return new File(artifactsDirectory, artifactName);
-  }
-
-  /**
-   * Returns the {@link File} which contains the {@link Manifest}.
-   *
-   * <p>The file may not exist.
-   */
-  public File getManifestFile() {
-    return new File(rootDirectory, "MANIFEST");
-  }
-
-  /**
-   * Returns the local location of this {@link LocalArtifactStagingLocation}.
-   *
-   * <p>This can be used to refer to the staging location when creating a retrieval service.
-   */
-  public String getRootPath() {
-    try {
-      return rootDirectory.getCanonicalPath();
-    } catch (IOException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactRetrievalService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactRetrievalService.java
deleted file mode 100644
index 41a3d8a..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactRetrievalService.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.artifact;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileChannel.MapMode;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactChunk;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.GetManifestResponse;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
-import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
-import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
-import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.stub.StreamObserver;
-
-/** An {@code ArtifactRetrievalService} which stages files to a local temp directory. */
-public class LocalFileSystemArtifactRetrievalService
-    extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase
-    implements ArtifactRetrievalService {
-  private static final int DEFAULT_CHUNK_SIZE = 2 * 1024 * 1024;
-
-  public static LocalFileSystemArtifactRetrievalService forRootDirectory(File base) {
-    return new LocalFileSystemArtifactRetrievalService(base);
-  }
-
-  private final LocalArtifactStagingLocation location;
-  private final Manifest manifest;
-
-  private LocalFileSystemArtifactRetrievalService(File rootDirectory) {
-    this.location = LocalArtifactStagingLocation.forExistingDirectory(rootDirectory);
-    try (FileInputStream manifestStream = new FileInputStream(location.getManifestFile())) {
-      this.manifest = ArtifactApi.Manifest.parseFrom(manifestStream);
-    } catch (FileNotFoundException e) {
-      throw new IllegalArgumentException(
-          String.format(
-              "No %s in root directory %s", Manifest.class.getSimpleName(), rootDirectory),
-          e);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public final void getManifest(
-      ArtifactApi.GetManifestRequest request,
-      StreamObserver<GetManifestResponse> responseObserver) {
-    try {
-      responseObserver.onNext(GetManifestResponse.newBuilder().setManifest(manifest).build());
-      responseObserver.onCompleted();
-    } catch (Exception e) {
-      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
-    }
-  }
-
-  /** Get the artifact with the provided name as a sequence of bytes. */
-  private ByteBuffer getArtifact(String name) throws IOException {
-    File artifact = location.getArtifactFile(name);
-    if (!artifact.exists()) {
-      throw new FileNotFoundException(String.format("No such artifact %s", name));
-    }
-    FileChannel input = new FileInputStream(artifact).getChannel();
-    return input.map(MapMode.READ_ONLY, 0L, input.size());
-  }
-
-  @Override
-  public void getArtifact(
-      ArtifactApi.GetArtifactRequest request,
-      StreamObserver<ArtifactApi.ArtifactChunk> responseObserver) {
-    try {
-      ByteBuffer artifact = getArtifact(request.getName());
-      do {
-        responseObserver.onNext(
-            ArtifactChunk.newBuilder()
-                .setData(
-                    ByteString.copyFrom(
-                        artifact, Math.min(artifact.remaining(), DEFAULT_CHUNK_SIZE)))
-                .build());
-      } while (artifact.hasRemaining());
-      responseObserver.onCompleted();
-    } catch (FileNotFoundException e) {
-      responseObserver.onError(
-          Status.INVALID_ARGUMENT
-              .withDescription(String.format("No such artifact %s", request.getName()))
-              .withCause(e)
-              .asException());
-    } catch (Exception e) {
-      responseObserver.onError(
-          Status.INTERNAL
-              .withDescription(
-                  String.format("Could not retrieve artifact with name %s", request.getName()))
-              .withCause(e)
-              .asException());
-    }
-  }
-
-  @Override
-  public void close() throws Exception {}
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactStagerService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactStagerService.java
deleted file mode 100644
index 3df1f54..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactStagerService.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.artifact;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
-import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc;
-import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.StatusException;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Throwables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An {@code ArtifactStagingService} which stages files to a local temp directory. TODO: refactor to
- * use staging session tokens
- */
-public class LocalFileSystemArtifactStagerService
-    extends ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase implements FnService {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(LocalFileSystemArtifactStagerService.class);
-
-  public static LocalFileSystemArtifactStagerService forRootDirectory(File base) {
-    return new LocalFileSystemArtifactStagerService(base);
-  }
-
-  private final LocalArtifactStagingLocation location;
-
-  private LocalFileSystemArtifactStagerService(File stagingBase) {
-    this.location = LocalArtifactStagingLocation.createAt(stagingBase);
-  }
-
-  @Override
-  public StreamObserver<ArtifactApi.PutArtifactRequest> putArtifact(
-      final StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver) {
-    return new CreateAndWriteFileObserver(responseObserver);
-  }
-
-  @Override
-  public void commitManifest(
-      ArtifactApi.CommitManifestRequest request,
-      StreamObserver<ArtifactApi.CommitManifestResponse> responseObserver) {
-    try {
-      commitManifestOrThrow(request, responseObserver);
-    } catch (StatusRuntimeException e) {
-      responseObserver.onError(e);
-      LOG.error("Failed to commit Manifest {}", request.getManifest(), e);
-    } catch (Exception e) {
-      responseObserver.onError(
-          Status.INTERNAL
-              .withCause(e)
-              .withDescription(Throwables.getStackTraceAsString(e))
-              .asRuntimeException());
-      LOG.error("Failed to commit Manifest {}", request.getManifest(), e);
-    }
-  }
-
-  private void commitManifestOrThrow(
-      ArtifactApi.CommitManifestRequest request,
-      StreamObserver<ArtifactApi.CommitManifestResponse> responseObserver)
-      throws IOException {
-    Collection<ArtifactApi.ArtifactMetadata> missing = new ArrayList<>();
-    for (ArtifactApi.ArtifactMetadata artifact : request.getManifest().getArtifactList()) {
-      // TODO: Validate the checksums on the server side, to fail more aggressively if require
-      if (!location.getArtifactFile(artifact.getName()).exists()) {
-        missing.add(artifact);
-      }
-    }
-    if (!missing.isEmpty()) {
-      throw Status.INVALID_ARGUMENT
-          .withDescription(
-              String.format("Attempted to commit manifest with missing Artifacts: [%s]", missing))
-          .asRuntimeException();
-    }
-    File mf = location.getManifestFile();
-    checkState(mf.createNewFile(), "Could not create file to store manifest");
-    try (OutputStream mfOut = new FileOutputStream(mf)) {
-      request.getManifest().writeTo(mfOut);
-    }
-    responseObserver.onNext(
-        ArtifactApi.CommitManifestResponse.newBuilder()
-            .setRetrievalToken(location.getRootPath())
-            .build());
-    responseObserver.onCompleted();
-  }
-
-  @Override
-  public void close() throws Exception {
-    // TODO: Close all active staging calls, signalling errors to the caller.
-  }
-
-  @VisibleForTesting
-  LocalArtifactStagingLocation getLocation() {
-    return location;
-  }
-
-  private class CreateAndWriteFileObserver
-      implements StreamObserver<ArtifactApi.PutArtifactRequest> {
-    private final StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver;
-    private FileWritingObserver writer;
-
-    private CreateAndWriteFileObserver(
-        StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver) {
-      this.responseObserver = responseObserver;
-    }
-
-    @Override
-    public void onNext(ArtifactApi.PutArtifactRequest value) {
-      try {
-        if (writer == null) {
-          if (!value.getContentCase().equals(ArtifactApi.PutArtifactRequest.ContentCase.METADATA)) {
-            throw Status.INVALID_ARGUMENT
-                .withDescription(
-                    String.format(
-                        "Expected the first %s to contain the Artifact Name, got %s",
-                        ArtifactApi.PutArtifactRequest.class.getSimpleName(),
-                        value.getContentCase()))
-                .asRuntimeException();
-          }
-          writer = createFile(value.getMetadata().getMetadata());
-        } else {
-          writer.onNext(value);
-        }
-      } catch (StatusRuntimeException e) {
-        responseObserver.onError(e);
-      } catch (Exception e) {
-        responseObserver.onError(
-            Status.INTERNAL
-                .withCause(e)
-                .withDescription(Throwables.getStackTraceAsString(e))
-                .asRuntimeException());
-      }
-    }
-
-    private FileWritingObserver createFile(ArtifactApi.ArtifactMetadata metadata)
-        throws IOException {
-      File destination = location.getArtifactFile(metadata.getName());
-      if (!destination.createNewFile()) {
-        throw Status.ALREADY_EXISTS
-            .withDescription(String.format("Artifact with name %s already exists", metadata))
-            .asRuntimeException();
-      }
-      return new FileWritingObserver(
-          destination, new FileOutputStream(destination), responseObserver);
-    }
-
-    @Override
-    public void onError(Throwable t) {
-      if (writer != null) {
-        writer.onError(t);
-      } else {
-        responseObserver.onCompleted();
-      }
-    }
-
-    @Override
-    public void onCompleted() {
-      if (writer != null) {
-        writer.onCompleted();
-      } else {
-        responseObserver.onCompleted();
-      }
-    }
-  }
-
-  private static class FileWritingObserver
-      implements StreamObserver<ArtifactApi.PutArtifactRequest> {
-    private final File destination;
-    private final OutputStream target;
-    private final StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver;
-
-    private FileWritingObserver(
-        File destination,
-        OutputStream target,
-        StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver) {
-      this.destination = destination;
-      this.target = target;
-      this.responseObserver = responseObserver;
-    }
-
-    @Override
-    public void onNext(ArtifactApi.PutArtifactRequest value) {
-      try {
-        if (value.getData() == null) {
-          StatusRuntimeException e =
-              Status.INVALID_ARGUMENT
-                  .withDescription(
-                      String.format(
-                          "Expected all chunks in the current stream state to contain data, got %s",
-                          value.getContentCase()))
-                  .asRuntimeException();
-          throw e;
-        }
-        value.getData().getData().writeTo(target);
-      } catch (Exception e) {
-        cleanedUp(e);
-      }
-    }
-
-    @Override
-    public void onError(Throwable t) {
-      if (cleanedUp(null)) {
-        responseObserver.onCompleted();
-      }
-    }
-
-    @Override
-    public void onCompleted() {
-      try {
-        target.close();
-      } catch (IOException e) {
-        LOG.error("Failed to complete writing file {}", destination, e);
-        cleanedUp(e);
-        return;
-      }
-      responseObserver.onNext(ArtifactApi.PutArtifactResponse.getDefaultInstance());
-      responseObserver.onCompleted();
-    }
-
-    /**
-     * Cleans up after the file writing failed exceptionally, due to an error either in the service
-     * or sent from the client.
-     *
-     * @return false if an error was reported, true otherwise
-     */
-    private boolean cleanedUp(@Nullable Throwable whyFailed) {
-      Throwable actual = whyFailed;
-      try {
-        target.close();
-        if (!destination.delete()) {
-          LOG.debug("Couldn't delete failed write at {}", destination);
-        }
-      } catch (IOException e) {
-        if (whyFailed == null) {
-          actual = e;
-        } else {
-          actual.addSuppressed(e);
-        }
-        LOG.error("Failed to clean up after writing file {}", destination, e);
-      }
-      if (actual != null) {
-        if (actual instanceof StatusException || actual instanceof StatusRuntimeException) {
-          responseObserver.onError(actual);
-        } else {
-          Status status =
-              Status.INTERNAL
-                  .withCause(actual)
-                  .withDescription(Throwables.getStackTraceAsString(actual));
-          responseObserver.onError(status.asException());
-        }
-      }
-      return actual == null;
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/UnsupportedArtifactRetrievalService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/UnsupportedArtifactRetrievalService.java
deleted file mode 100644
index f0d56ba..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/UnsupportedArtifactRetrievalService.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.artifact;
-
-import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
-import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
-
-/**
- * An {@link ArtifactRetrievalService} which has not implemented any methods.
- *
- * <p>For use with an in-process SDK harness.
- */
-public class UnsupportedArtifactRetrievalService
-    extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase
-    implements ArtifactRetrievalService {
-
-  public static ArtifactRetrievalService create() {
-    return new UnsupportedArtifactRetrievalService();
-  }
-
-  private UnsupportedArtifactRetrievalService() {}
-
-  @Override
-  public void close() {
-    // Do nothing.
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/package-info.java
deleted file mode 100644
index 1ec0da5..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/package-info.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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.
- */
-
-/** Provides local implementations of the Artifact API services. */
-package org.apache.beam.runners.direct.portable.artifact;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/PreparingJob.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/PreparingJob.java
deleted file mode 100644
index 077a517..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/PreparingJob.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.job;
-
-import com.google.auto.value.AutoValue;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactStagingService;
-import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.Struct;
-
-/** A Job with a {@code prepare} call but no corresponding {@code run} call. */
-@AutoValue
-abstract class PreparingJob implements AutoCloseable {
-  public static Builder builder() {
-    return new AutoValue_PreparingJob.Builder();
-  }
-
-  abstract Pipeline getPipeline();
-
-  abstract Struct getOptions();
-
-  abstract String getStagingSessionToken();
-
-  abstract GrpcFnServer<BeamFileSystemArtifactStagingService> getArtifactStagingServer();
-
-  @Override
-  public void close() throws Exception {
-    getArtifactStagingServer().close();
-  }
-
-  @AutoValue.Builder
-  public abstract static class Builder {
-    abstract Builder setPipeline(Pipeline pipeline);
-
-    abstract Builder setOptions(Struct options);
-
-    abstract Builder setStagingSessionToken(String stagingSessionToken);
-
-    abstract Builder setArtifactStagingServer(
-        GrpcFnServer<BeamFileSystemArtifactStagingService> server);
-
-    abstract PreparingJob build();
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobServer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobServer.java
deleted file mode 100644
index e2eac87..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobServer.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.job;
-
-import java.io.IOException;
-import java.nio.file.Paths;
-import java.util.Arrays;
-import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.ServerFactory;
-import org.kohsuke.args4j.CmdLineException;
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** A program that runs a {@link ReferenceRunnerJobService}. */
-public class ReferenceRunnerJobServer {
-  private static final Logger LOG = LoggerFactory.getLogger(ReferenceRunnerJobServer.class);
-  private final ServerConfiguration configuration;
-  private GrpcFnServer<ReferenceRunnerJobService> server;
-
-  private ReferenceRunnerJobServer(ServerConfiguration configuration) {
-    this.configuration = configuration;
-  }
-
-  public static void main(String[] args) throws Exception {
-    try {
-      runServer(parseConfiguration(args));
-    } catch (CmdLineException ignored) {
-    }
-  }
-
-  private static ServerConfiguration parseConfiguration(String[] args) throws CmdLineException {
-    ServerConfiguration configuration = new ServerConfiguration();
-    CmdLineParser parser = new CmdLineParser(configuration);
-    try {
-      parser.parseArgument(args);
-    } catch (CmdLineException e) {
-      e.printStackTrace(System.err);
-      printUsage(parser);
-      throw e;
-    }
-    return configuration;
-  }
-
-  private static void printUsage(CmdLineParser parser) {
-    System.err.println(
-        String.format(
-            "Usage: java %s arguments...", ReferenceRunnerJobService.class.getSimpleName()));
-    parser.printUsage(System.err);
-    System.err.println();
-  }
-
-  private static void runServer(ServerConfiguration configuration) throws Exception {
-    ServerFactory serverFactory = ServerFactory.createDefault();
-    ReferenceRunnerJobService.Configuration jobServiceConfig =
-        createJobServiceConfig(configuration);
-    ReferenceRunnerJobService service =
-        ReferenceRunnerJobService.create(serverFactory, jobServiceConfig);
-    try (GrpcFnServer<ReferenceRunnerJobService> server =
-        createServer(configuration, serverFactory, service)) {
-      System.out.println(
-          String.format(
-              "Started %s at %s",
-              ReferenceRunnerJobService.class.getSimpleName(),
-              server.getApiServiceDescriptor().getUrl()));
-      server.getServer().awaitTermination();
-    }
-    System.out.println("Server shut down, exiting");
-  }
-
-  public static ReferenceRunnerJobServer fromParams(String[] args) {
-    try {
-      return new ReferenceRunnerJobServer(parseConfiguration(args));
-    } catch (CmdLineException e) {
-      throw new IllegalArgumentException(
-          "Unable to parse command line arguments " + Arrays.asList(args), e);
-    }
-  }
-
-  public String start() throws Exception {
-    ServerFactory serverFactory = ServerFactory.createDefault();
-    ReferenceRunnerJobService.Configuration jobServiceConfig =
-        createJobServiceConfig(configuration);
-    server =
-        createServer(
-            configuration,
-            serverFactory,
-            ReferenceRunnerJobService.create(serverFactory, jobServiceConfig));
-
-    return server.getApiServiceDescriptor().getUrl();
-  }
-
-  public void stop() {
-    if (server != null) {
-      try {
-        server.close();
-      } catch (Exception e) {
-        LOG.error("Unable to stop job server.", e);
-      }
-    }
-  }
-
-  private static GrpcFnServer<ReferenceRunnerJobService> createServer(
-      ServerConfiguration configuration,
-      ServerFactory serverFactory,
-      ReferenceRunnerJobService service)
-      throws IOException {
-    if (configuration.port <= 0) {
-      return GrpcFnServer.allocatePortAndCreateFor(service, serverFactory);
-    }
-    return GrpcFnServer.create(
-        service,
-        ApiServiceDescriptor.newBuilder().setUrl("localhost:" + configuration.port).build(),
-        serverFactory);
-  }
-
-  /**
-   * Helper function to fill out a {@code ReferenceRunnerJobService.Configuration Configuration}
-   * object for {@code ReferenceRunnerJobService}.
-   */
-  private static ReferenceRunnerJobService.Configuration createJobServiceConfig(
-      ServerConfiguration configuration) {
-    ReferenceRunnerJobService.Configuration jobServiceConfig =
-        new ReferenceRunnerJobService.Configuration();
-    jobServiceConfig.artifactStagingPath = configuration.artifactStagingPath;
-    jobServiceConfig.keepArtifacts = configuration.keepArtifacts;
-    return jobServiceConfig;
-  }
-
-  /** Command-line options to configure the JobServer. */
-  public static class ServerConfiguration {
-    @Option(
-        name = "-p",
-        aliases = {"--port"},
-        usage = "The local port to expose the server on. 0 to use a dynamic port. (Default: 8099)")
-    private int port = 8099;
-
-    @Option(name = "--artifacts-dir", usage = "The location to store staged artifact files")
-    String artifactStagingPath =
-        Paths.get(System.getProperty("java.io.tmpdir"), "beam-artifact-staging").toString();
-
-    @Option(
-        name = "--keep-artifacts",
-        usage = "When enabled, do not delete staged artifacts when a job completes")
-    boolean keepArtifacts;
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
deleted file mode 100644
index 07517af..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
+++ /dev/null
@@ -1,308 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable.job;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.beam.model.jobmanagement.v1.JobApi;
-import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobState;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum;
-import org.apache.beam.model.jobmanagement.v1.JobApi.PrepareJobResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobResponse;
-import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceImplBase;
-import org.apache.beam.runners.direct.portable.ReferenceRunner;
-import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.ServerFactory;
-import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactStagingService;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This JobService implements the grpc calls for running jobs by using the {@code ReferenceRunner}
- * as an engine.
- */
-public class ReferenceRunnerJobService extends JobServiceImplBase implements FnService {
-
-  /** A configuration object for constructing the {@code ReferenceRunnerJobService}. */
-  public static class Configuration {
-    public String artifactStagingPath;
-    public boolean keepArtifacts;
-  }
-
-  private static final Logger LOG = LoggerFactory.getLogger(ReferenceRunnerJobService.class);
-  private static final int WAIT_MS = 1000;
-
-  public static ReferenceRunnerJobService create(
-      final ServerFactory serverFactory, Configuration configuration) {
-    LOG.info("Starting {}", ReferenceRunnerJobService.class);
-    return new ReferenceRunnerJobService(serverFactory, configuration);
-  }
-
-  private final ServerFactory serverFactory;
-  private final Configuration configuration;
-
-  private final ConcurrentMap<String, PreparingJob> unpreparedJobs;
-  private final ConcurrentMap<String, ReferenceRunner> runningJobs;
-  private final ConcurrentMap<String, JobState.Enum> jobStates;
-  private final ExecutorService executor;
-  private final ConcurrentLinkedQueue<GrpcFnServer<BeamFileSystemArtifactStagingService>>
-      artifactStagingServices;
-
-  private ReferenceRunnerJobService(ServerFactory serverFactory, Configuration configuration) {
-    this.serverFactory = serverFactory;
-    this.configuration = configuration;
-    unpreparedJobs = new ConcurrentHashMap<>();
-    runningJobs = new ConcurrentHashMap<>();
-    jobStates = new ConcurrentHashMap<>();
-    executor =
-        Executors.newCachedThreadPool(
-            new ThreadFactoryBuilder()
-                .setDaemon(false)
-                .setNameFormat("reference-runner-pipeline-%s")
-                .build());
-    artifactStagingServices = new ConcurrentLinkedQueue<>();
-  }
-
-  @Override
-  public void prepare(
-      JobApi.PrepareJobRequest request,
-      StreamObserver<JobApi.PrepareJobResponse> responseObserver) {
-    try {
-      LOG.trace("{} {}", PrepareJobResponse.class.getSimpleName(), request);
-
-      String preparationId = request.getJobName() + ThreadLocalRandom.current().nextInt();
-      GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingService =
-          createArtifactStagingService();
-      artifactStagingServices.add(artifactStagingService);
-      String stagingSessionToken =
-          BeamFileSystemArtifactStagingService.generateStagingSessionToken(
-              preparationId, configuration.artifactStagingPath);
-      PreparingJob existingJob =
-          unpreparedJobs.putIfAbsent(
-              preparationId,
-              PreparingJob.builder()
-                  .setArtifactStagingServer(artifactStagingService)
-                  .setPipeline(request.getPipeline())
-                  .setOptions(request.getPipelineOptions())
-                  .setStagingSessionToken(stagingSessionToken)
-                  .build());
-      checkArgument(
-          existingJob == null, "Unexpected existing job with preparation ID %s", preparationId);
-
-      responseObserver.onNext(
-          PrepareJobResponse.newBuilder()
-              .setPreparationId(preparationId)
-              .setArtifactStagingEndpoint(artifactStagingService.getApiServiceDescriptor())
-              .setStagingSessionToken(stagingSessionToken)
-              .build());
-      responseObserver.onCompleted();
-    } catch (Exception e) {
-      LOG.error("Could not prepare job with name {}", request.getJobName(), e);
-      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
-    }
-  }
-
-  private GrpcFnServer<BeamFileSystemArtifactStagingService> createArtifactStagingService()
-      throws Exception {
-    BeamFileSystemArtifactStagingService service = new BeamFileSystemArtifactStagingService();
-    return GrpcFnServer.allocatePortAndCreateFor(service, serverFactory);
-  }
-
-  @Override
-  @SuppressWarnings("FutureReturnValueIgnored") // Run API does not block on execution
-  public void run(
-      JobApi.RunJobRequest request, StreamObserver<JobApi.RunJobResponse> responseObserver) {
-    try {
-      LOG.trace("{} {}", RunJobRequest.class.getSimpleName(), request);
-      String preparationId = request.getPreparationId();
-      PreparingJob preparingJob = unpreparedJobs.get(preparationId);
-      if (preparingJob == null) {
-        responseObserver.onError(
-            Status.INVALID_ARGUMENT
-                .withDescription(String.format("Unknown Preparation Id %s", preparationId))
-                .asException());
-        return;
-      }
-      try {
-        // Close any preparation-time only resources.
-        preparingJob.close();
-      } catch (Exception e) {
-        responseObserver.onError(e);
-      }
-
-      ReferenceRunner runner =
-          ReferenceRunner.forPipeline(
-              preparingJob.getPipeline(), preparingJob.getOptions(), request.getRetrievalToken());
-      String jobId = "job-" + Integer.toString(ThreadLocalRandom.current().nextInt());
-      responseObserver.onNext(RunJobResponse.newBuilder().setJobId(jobId).build());
-      responseObserver.onCompleted();
-      runningJobs.put(jobId, runner);
-      jobStates.putIfAbsent(jobId, Enum.RUNNING);
-      executor.submit(
-          () -> {
-            try {
-              jobStates.computeIfPresent(jobId, (id, status) -> Enum.RUNNING);
-              runner.execute();
-              jobStates.computeIfPresent(jobId, (id, status) -> Enum.DONE);
-            } catch (Exception e) {
-              jobStates.computeIfPresent(jobId, (id, status) -> Enum.FAILED);
-              throw e;
-            }
-
-            // Delete artifacts after job is done.
-            if (!configuration.keepArtifacts) {
-              String stagingSessionToken = preparingJob.getStagingSessionToken();
-              try {
-                preparingJob
-                    .getArtifactStagingServer()
-                    .getService()
-                    .removeArtifacts(stagingSessionToken);
-              } catch (Exception e) {
-                LOG.error(
-                    "Failed to remove job staging directory for token {}: {}",
-                    stagingSessionToken,
-                    e);
-              }
-            }
-            return null;
-          });
-    } catch (StatusRuntimeException e) {
-      responseObserver.onError(e);
-    } catch (Exception e) {
-      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
-    }
-  }
-
-  @Override
-  public void getState(
-      GetJobStateRequest request, StreamObserver<GetJobStateResponse> responseObserver) {
-    LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
-    try {
-      responseObserver.onNext(
-          GetJobStateResponse.newBuilder()
-              .setState(jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED))
-              .build());
-      responseObserver.onCompleted();
-    } catch (Exception e) {
-      String errMessage =
-          String.format("Encountered Unexpected Exception for Invocation %s", request.getJobId());
-      LOG.error(errMessage, e);
-      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
-    }
-  }
-
-  @Override
-  public void getStateStream(
-      GetJobStateRequest request, StreamObserver<GetJobStateResponse> responseObserver) {
-    LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
-    String invocationId = request.getJobId();
-    try {
-      Thread.sleep(WAIT_MS);
-      Enum state = jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED);
-      responseObserver.onNext(GetJobStateResponse.newBuilder().setState(state).build());
-      while (Enum.RUNNING.equals(state)) {
-        Thread.sleep(WAIT_MS);
-        state = jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED);
-      }
-      responseObserver.onNext(GetJobStateResponse.newBuilder().setState(state).build());
-    } catch (Exception e) {
-      String errMessage =
-          String.format("Encountered Unexpected Exception for Invocation %s", invocationId);
-      LOG.error(errMessage, e);
-      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
-    }
-    responseObserver.onCompleted();
-  }
-
-  @Override
-  public void describePipelineOptions(
-      JobApi.DescribePipelineOptionsRequest request,
-      StreamObserver<JobApi.DescribePipelineOptionsResponse> responseObserver) {
-    LOG.trace("{} {}", JobApi.DescribePipelineOptionsRequest.class.getSimpleName(), request);
-    try {
-      JobApi.DescribePipelineOptionsResponse response =
-          JobApi.DescribePipelineOptionsResponse.newBuilder()
-              .addAllOptions(
-                  PipelineOptionsFactory.describe(PipelineOptionsFactory.getRegisteredOptions()))
-              .build();
-      responseObserver.onNext(response);
-      responseObserver.onCompleted();
-    } catch (Exception e) {
-      LOG.error("Error describing pipeline options", e);
-      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
-    }
-  }
-
-  @Override
-  public void getMessageStream(
-      JobMessagesRequest request, StreamObserver<JobMessagesResponse> responseObserver) {
-    // Not implemented
-    LOG.trace("{} {}", JobMessagesRequest.class.getSimpleName(), request);
-  }
-
-  @Override
-  public void cancel(CancelJobRequest request, StreamObserver<CancelJobResponse> responseObserver) {
-    LOG.trace("{} {}", CancelJobRequest.class.getSimpleName(), request);
-    responseObserver.onError(
-        Status.NOT_FOUND
-            .withDescription(String.format("Unknown Job ID %s", request.getJobId()))
-            .asException());
-  }
-
-  @Override
-  public void close() throws Exception {
-    for (PreparingJob preparingJob : ImmutableList.copyOf(unpreparedJobs.values())) {
-      try {
-        preparingJob.close();
-      } catch (Exception e) {
-        LOG.warn("Exception while closing preparing job {}", preparingJob);
-      }
-    }
-    while (!artifactStagingServices.isEmpty()) {
-      GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingService =
-          artifactStagingServices.remove();
-      try {
-        artifactStagingService.close();
-      } catch (Exception e) {
-        LOG.error(
-            "Unable to close staging sevice started on %s",
-            artifactStagingService.getApiServiceDescriptor().getUrl(), e);
-      }
-    }
-  }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/package-info.java
deleted file mode 100644
index 9de085b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * An execution engine for Beam Pipelines that uses the Java Runner harness and the Fn API to
- * execute.
- */
-package org.apache.beam.runners.direct.portable.job;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/package-info.java
deleted file mode 100644
index 1a51b2f..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/package-info.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Defines the {@link org.apache.beam.sdk.options.PipelineOptions.DirectRunner} which executes both
- * Bounded and Unbounded {@code Pipelines} on the local machine.
- *
- * <p>See {@link org.apache.beam.sdk.runners} for more information about Pipeline Runners.
- */
-package org.apache.beam.runners.direct.portable;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
index 06c8781..4c4ba94 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
@@ -21,7 +21,6 @@ import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
 import static org.junit.Assert.assertThat;
 
 import java.util.Set;
-import org.apache.beam.runners.direct.portable.ExecutableGraphBuilder;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.metrics.MetricResults;
@@ -65,7 +64,6 @@ public class DirectRunnerApiSurfaceTest {
             .pruningClass(DirectGraphs.class)
             .pruningClass(
                 WatermarkManager.class /* TODO: BEAM-4237 Consider moving to local-java */)
-            .pruningClass(ExecutableGraphBuilder.class)
             .pruningPattern(
                 "org[.]apache[.]beam[.]runners[.]direct[.]portable.*"
                 /* TODO: BEAM-4237 reconsider package layout with the ReferenceRunner */ )
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
deleted file mode 100644
index 0236aef..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components.Builder;
-import org.apache.beam.runners.core.construction.Environments;
-import org.apache.beam.runners.core.construction.RehydratedComponents;
-import org.apache.beam.runners.core.construction.SdkComponents;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link BundleFactoryOutputReceiverFactory}. */
-@RunWith(JUnit4.class)
-public class BundleFactoryOutputReceiverFactoryTest {
-  private final BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-  private PCollectionNode fooPC;
-  private PCollectionNode barPC;
-  private RunnerApi.Components baseComponents;
-
-  private OutputReceiverFactory factory;
-  private Collection<UncommittedBundle<?>> outputBundles;
-
-  @Before
-  public void setup() throws IOException {
-    Pipeline p = Pipeline.create();
-    PCollection<String> foo =
-        p.apply("createFoo", Create.of("1", "2", "3"))
-            .apply("windowFoo", Window.into(FixedWindows.of(Duration.standardMinutes(5L))));
-    PCollection<Integer> bar = p.apply("bar", Create.of(1, 2, 3));
-
-    SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
-    String fooId = sdkComponents.registerPCollection(foo);
-    String barId = sdkComponents.registerPCollection(bar);
-    baseComponents = sdkComponents.toComponents();
-
-    fooPC = PipelineNode.pCollection(fooId, baseComponents.getPcollectionsOrThrow(fooId));
-    barPC = PipelineNode.pCollection(barId, baseComponents.getPcollectionsOrThrow(barId));
-
-    outputBundles = new ArrayList<>();
-    factory =
-        BundleFactoryOutputReceiverFactory.create(
-            bundleFactory, baseComponents, outputBundles::add);
-  }
-
-  @Test
-  public void addsBundlesToResult() {
-    factory.create(fooPC.getId());
-    factory.create(barPC.getId());
-
-    assertThat(Iterables.size(outputBundles), equalTo(2));
-
-    Collection<PCollectionNode> pcollections = new ArrayList<>();
-    for (UncommittedBundle<?> bundle : outputBundles) {
-      pcollections.add(bundle.getPCollection());
-    }
-    assertThat(pcollections, containsInAnyOrder(fooPC, barPC));
-  }
-
-  @Test
-  public void receiverAddsElementsToBundle() throws Exception {
-    FnDataReceiver<WindowedValue<byte[]>> receiver = factory.create(fooPC.getId());
-
-    Builder builder = baseComponents.toBuilder();
-    String sdkWireCoderId = WireCoders.addSdkWireCoder(fooPC, builder);
-    Components components = builder.build();
-
-    Coder<WindowedValue<String>> sdkCoder =
-        (Coder<WindowedValue<String>>)
-            RehydratedComponents.forComponents(components).getCoder(sdkWireCoderId);
-    Coder<WindowedValue<byte[]>> runnerCoder =
-        WireCoders.instantiateRunnerWireCoder(fooPC, components);
-
-    WindowedValue<byte[]> firstElem =
-        CoderUtils.decodeFromByteArray(
-            runnerCoder,
-            CoderUtils.encodeToByteArray(
-                sdkCoder,
-                WindowedValue.of(
-                    "1",
-                    new Instant(120),
-                    new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
-                    PaneInfo.NO_FIRING)));
-    WindowedValue<byte[]> secondElem =
-        CoderUtils.decodeFromByteArray(
-            runnerCoder,
-            CoderUtils.encodeToByteArray(
-                sdkCoder,
-                WindowedValue.of(
-                    "2",
-                    new Instant(240),
-                    new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
-                    PaneInfo.NO_FIRING)));
-    receiver.accept(firstElem);
-    receiver.accept(secondElem);
-
-    CommittedBundle<?> output = getOnlyElement(outputBundles).commit(Instant.now());
-    assertThat(output, containsInAnyOrder(firstElem, secondElem));
-  }
-
-  /**
-   * Tests that if a {@link
-   * org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode} is provided
-   * multiple times, the returned {@link
-   * org.apache.beam.runners.fnexecution.control.RemoteOutputReceiver} instances are independent.
-   */
-  @Test
-  public void multipleInstancesOfPCollectionIndependent() throws Exception {
-    FnDataReceiver<WindowedValue<byte[]>> firstReceiver = factory.create(fooPC.getId());
-    FnDataReceiver<WindowedValue<byte[]>> secondReceiver = factory.create(fooPC.getId());
-
-    Components.Builder builder = baseComponents.toBuilder();
-    String sdkWireCoderId = WireCoders.addSdkWireCoder(fooPC, builder);
-    Components components = builder.build();
-
-    Coder<WindowedValue<String>> sdkCoder =
-        (Coder<WindowedValue<String>>)
-            RehydratedComponents.forComponents(components).getCoder(sdkWireCoderId);
-
-    Coder<WindowedValue<byte[]>> runnerCoder =
-        WireCoders.instantiateRunnerWireCoder(fooPC, components);
-
-    WindowedValue<byte[]> firstElem =
-        CoderUtils.decodeFromByteArray(
-            runnerCoder,
-            CoderUtils.encodeToByteArray(
-                sdkCoder,
-                WindowedValue.of(
-                    "1",
-                    new Instant(120),
-                    new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
-                    PaneInfo.NO_FIRING)));
-    firstReceiver.accept(firstElem);
-
-    WindowedValue<byte[]> secondElem =
-        CoderUtils.decodeFromByteArray(
-            runnerCoder,
-            CoderUtils.encodeToByteArray(
-                sdkCoder,
-                WindowedValue.of(
-                    "2",
-                    new Instant(240),
-                    new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
-                    PaneInfo.NO_FIRING)));
-    secondReceiver.accept(secondElem);
-
-    Collection<WindowedValue<?>> outputs = new ArrayList<>();
-    for (UncommittedBundle<?> uncommitted : outputBundles) {
-      assertThat(uncommitted.getPCollection(), equalTo(fooPC));
-      Iterable<? extends WindowedValue<?>> elements =
-          uncommitted.commit(Instant.now()).getElements();
-      Iterables.addAll(outputs, elements);
-      assertThat(Iterables.size(elements), equalTo(1));
-    }
-    assertThat(outputs, containsInAnyOrder(firstElem, secondElem));
-  }
-
-  @Test
-  public void differentPCollectionsIndependent() throws Exception {
-    FnDataReceiver<WindowedValue<byte[]>> fooReceiver = factory.create(fooPC.getId());
-
-    Components.Builder builder = baseComponents.toBuilder();
-    String sdkWireCoderId = WireCoders.addSdkWireCoder(fooPC, builder);
-    String barSdkWireCoderId = WireCoders.addSdkWireCoder(barPC, builder);
-    Components components = builder.build();
-
-    Coder<WindowedValue<String>> fooSdkCoder =
-        (Coder<WindowedValue<String>>)
-            RehydratedComponents.forComponents(components).getCoder(sdkWireCoderId);
-    Coder<WindowedValue<byte[]>> fooRunnerCoder =
-        WireCoders.instantiateRunnerWireCoder(fooPC, components);
-
-    FnDataReceiver<WindowedValue<byte[]>> barReceiver = factory.create(barPC.getId());
-    Coder<WindowedValue<Integer>> barSdkCoder =
-        (Coder<WindowedValue<Integer>>)
-            RehydratedComponents.forComponents(components).getCoder(barSdkWireCoderId);
-    Coder<WindowedValue<byte[]>> barRunnerCoder =
-        WireCoders.instantiateRunnerWireCoder(barPC, components);
-
-    WindowedValue<byte[]> fooElem =
-        CoderUtils.decodeFromByteArray(
-            fooRunnerCoder,
-            CoderUtils.encodeToByteArray(
-                fooSdkCoder,
-                WindowedValue.of(
-                    "1",
-                    new Instant(120),
-                    new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
-                    PaneInfo.NO_FIRING)));
-    fooReceiver.accept(fooElem);
-
-    WindowedValue<byte[]> barElem =
-        CoderUtils.decodeFromByteArray(
-            barRunnerCoder,
-            CoderUtils.encodeToByteArray(
-                barSdkCoder, WindowedValue.timestampedValueInGlobalWindow(2, new Instant(240))));
-    barReceiver.accept(barElem);
-
-    Collection<? super WindowedValue<?>> outputs = new ArrayList<>();
-    for (UncommittedBundle<?> uncommitted : outputBundles) {
-      WindowedValue<?> output = getOnlyElement(uncommitted.commit(Instant.now()).getElements());
-      if (fooPC.equals(uncommitted.getPCollection())) {
-        assertThat(output, equalTo(fooElem));
-      } else if (barPC.equals(uncommitted.getPCollection())) {
-        assertThat(output, equalTo(barElem));
-      } else {
-        fail(
-            String.format(
-                "Output %s should be either 'foo' or 'bar', got '%s",
-                PCollection.class.getSimpleName(), uncommitted.getPCollection().getId()));
-      }
-      outputs.add(output);
-    }
-    assertThat(outputs, containsInAnyOrder(fooElem, barElem));
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CommittedResultTest.java
deleted file mode 100644
index 357efb3..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CommittedResultTest.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.junit.Assert.assertThat;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.List;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.IsBounded.Enum;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link CommittedResult}. */
-@RunWith(JUnit4.class)
-public class CommittedResultTest implements Serializable {
-
-  @Rule
-  public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
-
-  private transient PCollectionNode created =
-      PipelineNode.pCollection(
-          "created", RunnerApi.PCollection.newBuilder().setUniqueName("created").build());
-  private transient PTransformNode transform =
-      PipelineNode.pTransform("foo", RunnerApi.PTransform.getDefaultInstance());
-  private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-
-  @Test
-  public void getTransformExtractsFromResult() {
-    CommittedResult<PTransformNode> result =
-        CommittedResult.create(
-            StepTransformResult.withoutHold(transform).build(),
-            Optional.absent(),
-            Collections.emptyList(),
-            EnumSet.noneOf(OutputType.class));
-
-    assertThat(result.getExecutable(), Matchers.equalTo(transform));
-  }
-
-  @Test
-  public void getUncommittedElementsEqualInput() {
-    CommittedBundle<Integer> bundle =
-        bundleFactory
-            .<Integer>createBundle(created)
-            .add(WindowedValue.valueInGlobalWindow(2))
-            .commit(Instant.now());
-    CommittedResult<PTransformNode> result =
-        CommittedResult.create(
-            StepTransformResult.withoutHold(transform).build(),
-            Optional.of(bundle),
-            Collections.emptyList(),
-            EnumSet.noneOf(OutputType.class));
-
-    assertThat(result.getUnprocessedInputs().get(), Matchers.equalTo(bundle));
-  }
-
-  @Test
-  public void getUncommittedElementsNull() {
-    CommittedResult<PTransformNode> result =
-        CommittedResult.create(
-            StepTransformResult.withoutHold(transform).build(),
-            Optional.absent(),
-            Collections.emptyList(),
-            EnumSet.noneOf(OutputType.class));
-
-    assertThat(result.getUnprocessedInputs(), Matchers.equalTo(Optional.absent()));
-  }
-
-  @Test
-  public void getOutputsEqualInput() {
-    List<? extends CommittedBundle<Integer>> outputs =
-        ImmutableList.of(
-            bundleFactory
-                .<Integer>createBundle(
-                    PipelineNode.pCollection(
-                        "bounded",
-                        RunnerApi.PCollection.newBuilder()
-                            .setUniqueName("bounded")
-                            .setIsBounded(Enum.BOUNDED)
-                            .build()))
-                .commit(Instant.now()),
-            bundleFactory
-                .<Integer>createBundle(
-                    PipelineNode.pCollection(
-                        "unbounded",
-                        RunnerApi.PCollection.newBuilder()
-                            .setUniqueName("unbounded")
-                            .setIsBounded(Enum.UNBOUNDED)
-                            .build()))
-                .commit(Instant.now()));
-    CommittedResult<PTransformNode> result =
-        CommittedResult.create(
-            StepTransformResult.withoutHold(transform).build(),
-            Optional.absent(),
-            outputs,
-            EnumSet.of(OutputType.BUNDLE, OutputType.PCOLLECTION_VIEW));
-
-    assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternalsTest.java
deleted file mode 100644
index 125860a..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternalsTest.java
+++ /dev/null
@@ -1,591 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.nullValue;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaceForTest;
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.CombiningState;
-import org.apache.beam.sdk.state.GroupingState;
-import org.apache.beam.sdk.state.MapState;
-import org.apache.beam.sdk.state.SetState;
-import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.state.WatermarkHoldState;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Lists;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link CopyOnAccessInMemoryStateInternals}. */
-@RunWith(JUnit4.class)
-public class CopyOnAccessInMemoryStateInternalsTest {
-
-  @Rule public final TestPipeline pipeline = TestPipeline.create();
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  private String key = "foo";
-
-  @Test
-  public void testGetWithEmpty() {
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = internals.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-    assertThat(stringBag.read(), containsInAnyOrder("baz", "bar"));
-
-    BagState<String> reReadStringBag = internals.state(namespace, bagTag);
-    assertThat(reReadStringBag.read(), containsInAnyOrder("baz", "bar"));
-  }
-
-  @Test
-  public void testGetWithAbsentInUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = internals.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-    assertThat(stringBag.read(), containsInAnyOrder("baz", "bar"));
-
-    BagState<String> reReadVoidBag = internals.state(namespace, bagTag);
-    assertThat(reReadVoidBag.read(), containsInAnyOrder("baz", "bar"));
-
-    BagState<String> underlyingState = underlying.state(namespace, bagTag);
-    assertThat(underlyingState.read(), emptyIterable());
-  }
-
-  /**
-   * Tests that retrieving state with an underlying StateInternals with an existing value returns a
-   * value that initially has equal value to the provided state but can be modified without
-   * modifying the existing state.
-   */
-  @Test
-  public void testGetWithPresentInUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<ValueState<String>> valueTag = StateTags.value("foo", StringUtf8Coder.of());
-    ValueState<String> underlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.read(), nullValue(String.class));
-
-    underlyingValue.write("bar");
-    assertThat(underlyingValue.read(), equalTo("bar"));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-    ValueState<String> copyOnAccessState = internals.state(namespace, valueTag);
-    assertThat(copyOnAccessState.read(), equalTo("bar"));
-
-    copyOnAccessState.write("baz");
-    assertThat(copyOnAccessState.read(), equalTo("baz"));
-    assertThat(underlyingValue.read(), equalTo("bar"));
-
-    ValueState<String> reReadUnderlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
-  }
-
-  @Test
-  public void testBagStateWithUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<Integer>> valueTag = StateTags.bag("foo", VarIntCoder.of());
-    BagState<Integer> underlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.read(), emptyIterable());
-
-    underlyingValue.add(1);
-    assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-    BagState<Integer> copyOnAccessState = internals.state(namespace, valueTag);
-    assertThat(copyOnAccessState.read(), containsInAnyOrder(1));
-
-    copyOnAccessState.add(4);
-    assertThat(copyOnAccessState.read(), containsInAnyOrder(4, 1));
-    assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
-    BagState<Integer> reReadUnderlyingValue = underlying.state(namespace, valueTag);
-    assertThat(
-        Lists.newArrayList(underlyingValue.read()),
-        equalTo(Lists.newArrayList(reReadUnderlyingValue.read())));
-  }
-
-  @Test
-  public void testSetStateWithUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<SetState<Integer>> valueTag = StateTags.set("foo", VarIntCoder.of());
-    SetState<Integer> underlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.read(), emptyIterable());
-
-    underlyingValue.add(1);
-    assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-    SetState<Integer> copyOnAccessState = internals.state(namespace, valueTag);
-    assertThat(copyOnAccessState.read(), containsInAnyOrder(1));
-
-    copyOnAccessState.add(4);
-    assertThat(copyOnAccessState.read(), containsInAnyOrder(4, 1));
-    assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
-    SetState<Integer> reReadUnderlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
-  }
-
-  @Test
-  public void testMapStateWithUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<MapState<String, Integer>> valueTag =
-        StateTags.map("foo", StringUtf8Coder.of(), VarIntCoder.of());
-    MapState<String, Integer> underlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.entries().read(), emptyIterable());
-
-    underlyingValue.put("hello", 1);
-    assertThat(underlyingValue.get("hello").read(), equalTo(1));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-    MapState<String, Integer> copyOnAccessState = internals.state(namespace, valueTag);
-    assertThat(copyOnAccessState.get("hello").read(), equalTo(1));
-
-    copyOnAccessState.put("world", 4);
-    assertThat(copyOnAccessState.get("hello").read(), equalTo(1));
-    assertThat(copyOnAccessState.get("world").read(), equalTo(4));
-    assertThat(underlyingValue.get("hello").read(), equalTo(1));
-    assertNull(underlyingValue.get("world").read());
-
-    MapState<String, Integer> reReadUnderlyingValue = underlying.state(namespace, valueTag);
-    assertThat(underlyingValue.entries().read(), equalTo(reReadUnderlyingValue.entries().read()));
-  }
-
-  @Test
-  public void testAccumulatorCombiningStateWithUnderlying() throws CannotProvideCoderException {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CombineFn<Long, long[], Long> sumLongFn = Sum.ofLongs();
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    CoderRegistry reg = pipeline.getCoderRegistry();
-    StateTag<CombiningState<Long, long[], Long>> stateTag =
-        StateTags.combiningValue(
-            "summer", sumLongFn.getAccumulatorCoder(reg, reg.getCoder(Long.class)), sumLongFn);
-    GroupingState<Long, Long> underlyingValue = underlying.state(namespace, stateTag);
-    assertThat(underlyingValue.read(), equalTo(0L));
-
-    underlyingValue.add(1L);
-    assertThat(underlyingValue.read(), equalTo(1L));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-    GroupingState<Long, Long> copyOnAccessState = internals.state(namespace, stateTag);
-    assertThat(copyOnAccessState.read(), equalTo(1L));
-
-    copyOnAccessState.add(4L);
-    assertThat(copyOnAccessState.read(), equalTo(5L));
-    assertThat(underlyingValue.read(), equalTo(1L));
-
-    GroupingState<Long, Long> reReadUnderlyingValue = underlying.state(namespace, stateTag);
-    assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
-  }
-
-  @Test
-  public void testWatermarkHoldStateWithUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    TimestampCombiner timestampCombiner = TimestampCombiner.EARLIEST;
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<WatermarkHoldState> stateTag =
-        StateTags.watermarkStateInternal("wmstate", timestampCombiner);
-    WatermarkHoldState underlyingValue = underlying.state(namespace, stateTag);
-    assertThat(underlyingValue.read(), nullValue());
-
-    underlyingValue.add(new Instant(250L));
-    assertThat(underlyingValue.read(), equalTo(new Instant(250L)));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-    WatermarkHoldState copyOnAccessState = internals.state(namespace, stateTag);
-    assertThat(copyOnAccessState.read(), equalTo(new Instant(250L)));
-
-    copyOnAccessState.add(new Instant(100L));
-    assertThat(copyOnAccessState.read(), equalTo(new Instant(100L)));
-    assertThat(underlyingValue.read(), equalTo(new Instant(250L)));
-
-    copyOnAccessState.add(new Instant(500L));
-    assertThat(copyOnAccessState.read(), equalTo(new Instant(100L)));
-
-    WatermarkHoldState reReadUnderlyingValue = underlying.state(namespace, stateTag);
-    assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
-  }
-
-  @Test
-  public void testCommitWithoutUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = internals.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-    assertThat(stringBag.read(), containsInAnyOrder("baz", "bar"));
-
-    internals.commit();
-
-    BagState<String> reReadStringBag = internals.state(namespace, bagTag);
-    assertThat(reReadStringBag.read(), containsInAnyOrder("baz", "bar"));
-    assertThat(internals.isEmpty(), is(false));
-  }
-
-  @Test
-  public void testCommitWithUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = underlying.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-
-    internals.commit();
-    BagState<String> reReadStringBag = internals.state(namespace, bagTag);
-    assertThat(reReadStringBag.read(), containsInAnyOrder("baz", "bar"));
-
-    reReadStringBag.add("spam");
-
-    BagState<String> underlyingState = underlying.state(namespace, bagTag);
-    assertThat(underlyingState.read(), containsInAnyOrder("spam", "bar", "baz"));
-    assertThat(underlyingState, is(theInstance(stringBag)));
-    assertThat(internals.isEmpty(), is(false));
-  }
-
-  @Test
-  public void testCommitWithClearedInUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CopyOnAccessInMemoryStateInternals<String> secondUnderlying =
-        spy(CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying));
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, secondUnderlying);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = underlying.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-    stringBag.clear();
-    // We should not read through the cleared bag
-    secondUnderlying.commit();
-
-    // Should not be visible
-    stringBag.add("foo");
-
-    internals.commit();
-    BagState<String> internalsStringBag = internals.state(namespace, bagTag);
-    assertThat(internalsStringBag.read(), emptyIterable());
-    verify(secondUnderlying, never()).state(namespace, bagTag);
-    assertThat(internals.isEmpty(), is(false));
-  }
-
-  @Test
-  public void testCommitWithOverwrittenUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = underlying.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-
-    BagState<String> internalsState = internals.state(namespace, bagTag);
-    internalsState.add("eggs");
-    internalsState.add("ham");
-    internalsState.add("0x00ff00");
-    internalsState.add("&");
-
-    internals.commit();
-
-    BagState<String> reReadInternalState = internals.state(namespace, bagTag);
-    assertThat(
-        reReadInternalState.read(),
-        containsInAnyOrder("bar", "baz", "0x00ff00", "eggs", "&", "ham"));
-    BagState<String> reReadUnderlyingState = underlying.state(namespace, bagTag);
-    assertThat(reReadUnderlyingState.read(), containsInAnyOrder("bar", "baz"));
-  }
-
-  @Test
-  public void testCommitWithAddedUnderlying() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
-    internals.commit();
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = underlying.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-
-    BagState<String> internalState = internals.state(namespace, bagTag);
-    assertThat(internalState.read(), emptyIterable());
-
-    BagState<String> reReadUnderlyingState = underlying.state(namespace, bagTag);
-    assertThat(reReadUnderlyingState.read(), containsInAnyOrder("bar", "baz"));
-  }
-
-  @Test
-  public void testCommitWithEmptyTableIsEmpty() {
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    internals.commit();
-
-    assertThat(internals.isEmpty(), is(true));
-  }
-
-  @Test
-  public void testCommitWithOnlyClearedValuesIsEmpty() {
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = internals.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("foo");
-    stringBag.clear();
-
-    internals.commit();
-
-    assertThat(internals.isEmpty(), is(true));
-  }
-
-  @Test
-  public void testCommitWithEmptyNewAndFullUnderlyingIsNotEmpty() {
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
-    StateNamespace namespace = new StateNamespaceForTest("foo");
-    StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
-    BagState<String> stringBag = underlying.state(namespace, bagTag);
-    assertThat(stringBag.read(), emptyIterable());
-
-    stringBag.add("bar");
-    stringBag.add("baz");
-
-    internals.commit();
-    assertThat(internals.isEmpty(), is(false));
-  }
-
-  @Test
-  public void testGetEarliestWatermarkHoldAfterCommit() {
-    BoundedWindow first =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(2048L);
-          }
-        };
-    BoundedWindow second =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(689743L);
-          }
-        };
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying("foo", null);
-
-    StateTag<WatermarkHoldState> firstHoldAddress =
-        StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
-    WatermarkHoldState firstHold =
-        internals.state(StateNamespaces.window(null, first), firstHoldAddress);
-    firstHold.add(new Instant(22L));
-
-    StateTag<WatermarkHoldState> secondHoldAddress =
-        StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
-    WatermarkHoldState secondHold =
-        internals.state(StateNamespaces.window(null, second), secondHoldAddress);
-    secondHold.add(new Instant(2L));
-
-    internals.commit();
-    assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(2L)));
-  }
-
-  @Test
-  public void testGetEarliestWatermarkHoldWithEarliestInUnderlyingTable() {
-    BoundedWindow first =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(2048L);
-          }
-        };
-    BoundedWindow second =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(689743L);
-          }
-        };
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying("foo", null);
-    StateTag<WatermarkHoldState> firstHoldAddress =
-        StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
-    WatermarkHoldState firstHold =
-        underlying.state(StateNamespaces.window(null, first), firstHoldAddress);
-    firstHold.add(new Instant(22L));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying("foo", underlying.commit());
-
-    StateTag<WatermarkHoldState> secondHoldAddress =
-        StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
-    WatermarkHoldState secondHold =
-        internals.state(StateNamespaces.window(null, second), secondHoldAddress);
-    secondHold.add(new Instant(244L));
-
-    internals.commit();
-    assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(22L)));
-  }
-
-  @Test
-  public void testGetEarliestWatermarkHoldWithEarliestInNewTable() {
-    BoundedWindow first =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(2048L);
-          }
-        };
-    BoundedWindow second =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(689743L);
-          }
-        };
-    CopyOnAccessInMemoryStateInternals<String> underlying =
-        CopyOnAccessInMemoryStateInternals.withUnderlying("foo", null);
-    StateTag<WatermarkHoldState> firstHoldAddress =
-        StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
-    WatermarkHoldState firstHold =
-        underlying.state(StateNamespaces.window(null, first), firstHoldAddress);
-    firstHold.add(new Instant(224L));
-
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying("foo", underlying.commit());
-
-    StateTag<WatermarkHoldState> secondHoldAddress =
-        StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
-    WatermarkHoldState secondHold =
-        internals.state(StateNamespaces.window(null, second), secondHoldAddress);
-    secondHold.add(new Instant(24L));
-
-    internals.commit();
-    assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(24L)));
-  }
-
-  @Test
-  public void testGetEarliestHoldBeforeCommit() {
-    CopyOnAccessInMemoryStateInternals<String> internals =
-        CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
-    internals
-        .state(
-            StateNamespaces.global(),
-            StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST))
-        .add(new Instant(1234L));
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(CopyOnAccessInMemoryStateInternals.class.getSimpleName());
-    thrown.expectMessage("Can't get the earliest watermark hold");
-    thrown.expectMessage("before it is committed");
-
-    internals.getEarliestWatermarkHold();
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectMetricsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectMetricsTest.java
deleted file mode 100644
index 59faca3..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectMetricsTest.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.sdk.metrics.MetricNameFilter.inNamespace;
-import static org.apache.beam.sdk.metrics.MetricResultsMatchers.attemptedMetricsResult;
-import static org.apache.beam.sdk.metrics.MetricResultsMatchers.committedMetricsResult;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.runners.core.metrics.DistributionData;
-import org.apache.beam.runners.core.metrics.GaugeData;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.GaugeResult;
-import org.apache.beam.sdk.metrics.MetricKey;
-import org.apache.beam.sdk.metrics.MetricName;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DirectMetrics}. */
-@RunWith(JUnit4.class)
-public class DirectMetricsTest {
-
-  @Mock private CommittedBundle<Object> bundle1;
-  @Mock private CommittedBundle<Object> bundle2;
-
-  private static final MetricName NAME1 = MetricName.named("ns1", "name1");
-  private static final MetricName NAME2 = MetricName.named("ns1", "name2");
-  private static final MetricName NAME3 = MetricName.named("ns2", "name1");
-  private static final MetricName NAME4 = MetricName.named("ns2", "name2");
-
-  private DirectMetrics metrics = new DirectMetrics();
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testApplyCommittedNoFilter() {
-    metrics.commitLogical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("step1", NAME1), 5L),
-                MetricUpdate.create(MetricKey.create("step1", NAME2), 8L)),
-            ImmutableList.of(
-                MetricUpdate.create(
-                    MetricKey.create("step1", NAME1), DistributionData.create(8, 2, 3, 5))),
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("step1", NAME4), GaugeData.create(15L)))));
-    metrics.commitLogical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("step2", NAME1), 7L),
-                MetricUpdate.create(MetricKey.create("step1", NAME2), 4L)),
-            ImmutableList.of(
-                MetricUpdate.create(
-                    MetricKey.create("step1", NAME1), DistributionData.create(4, 1, 4, 4))),
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("step1", NAME4), GaugeData.create(27L)))));
-
-    MetricQueryResults results = metrics.allMetrics();
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            attemptedMetricsResult("ns1", "name1", "step1", 0L),
-            attemptedMetricsResult("ns1", "name2", "step1", 0L),
-            attemptedMetricsResult("ns1", "name1", "step2", 0L)));
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            committedMetricsResult("ns1", "name1", "step1", 5L),
-            committedMetricsResult("ns1", "name2", "step1", 12L),
-            committedMetricsResult("ns1", "name1", "step2", 7L)));
-    assertThat(
-        results.getDistributions(),
-        contains(
-            attemptedMetricsResult("ns1", "name1", "step1", DistributionResult.IDENTITY_ELEMENT)));
-    assertThat(
-        results.getDistributions(),
-        contains(
-            committedMetricsResult(
-                "ns1", "name1", "step1", DistributionResult.create(12, 3, 3, 5))));
-    assertThat(
-        results.getGauges(),
-        contains(attemptedMetricsResult("ns2", "name2", "step1", GaugeResult.empty())));
-    assertThat(
-        results.getGauges(),
-        contains(
-            committedMetricsResult(
-                "ns2", "name2", "step1", GaugeResult.create(27L, Instant.now()))));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testApplyAttemptedCountersQueryOneNamespace() {
-    metrics.updatePhysical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("step1", NAME1), 5L),
-                MetricUpdate.create(MetricKey.create("step1", NAME3), 8L)),
-            ImmutableList.of(),
-            ImmutableList.of()));
-    metrics.updatePhysical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("step2", NAME1), 7L),
-                MetricUpdate.create(MetricKey.create("step1", NAME3), 4L)),
-            ImmutableList.of(),
-            ImmutableList.of()));
-
-    MetricQueryResults results =
-        metrics.queryMetrics(MetricsFilter.builder().addNameFilter(inNamespace("ns1")).build());
-
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            attemptedMetricsResult("ns1", "name1", "step1", 5L),
-            attemptedMetricsResult("ns1", "name1", "step2", 7L)));
-
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            committedMetricsResult("ns1", "name1", "step1", 0L),
-            committedMetricsResult("ns1", "name1", "step2", 0L)));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testApplyAttemptedQueryCompositeScope() {
-    metrics.updatePhysical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("Outer1/Inner1", NAME1), 5L),
-                MetricUpdate.create(MetricKey.create("Outer1/Inner2", NAME1), 8L)),
-            ImmutableList.of(),
-            ImmutableList.of()));
-    metrics.updatePhysical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("Outer1/Inner1", NAME1), 12L),
-                MetricUpdate.create(MetricKey.create("Outer2/Inner2", NAME1), 18L)),
-            ImmutableList.of(),
-            ImmutableList.of()));
-
-    MetricQueryResults results =
-        metrics.queryMetrics(MetricsFilter.builder().addStep("Outer1").build());
-
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            attemptedMetricsResult("ns1", "name1", "Outer1/Inner1", 12L),
-            attemptedMetricsResult("ns1", "name1", "Outer1/Inner2", 8L)));
-
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            committedMetricsResult("ns1", "name1", "Outer1/Inner1", 0L),
-            committedMetricsResult("ns1", "name1", "Outer1/Inner2", 0L)));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testPartialScopeMatchingInMetricsQuery() {
-    metrics.updatePhysical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner1", NAME1), 5L),
-                MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner2", NAME1), 8L)),
-            ImmutableList.of(),
-            ImmutableList.of()));
-    metrics.updatePhysical(
-        bundle1,
-        MetricUpdates.create(
-            ImmutableList.of(
-                MetricUpdate.create(MetricKey.create("Top2/Outer1/Inner1", NAME1), 12L),
-                MetricUpdate.create(MetricKey.create("Top1/Outer2/Inner2", NAME1), 18L)),
-            ImmutableList.of(),
-            ImmutableList.of()));
-
-    MetricQueryResults results =
-        metrics.queryMetrics(MetricsFilter.builder().addStep("Top1/Outer1").build());
-
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner1", 5L),
-            attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner2", 8L)));
-
-    results = metrics.queryMetrics(MetricsFilter.builder().addStep("Inner2").build());
-
-    assertThat(
-        results.getCounters(),
-        containsInAnyOrder(
-            attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner2", 8L),
-            attemptedMetricsResult("ns1", "name1", "Top1/Outer2/Inner2", 18L)));
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTimerInternalsTest.java
deleted file mode 100644
index f5c73fd..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTimerInternalsTest.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DirectTimerInternals}. */
-@RunWith(JUnit4.class)
-public class DirectTimerInternalsTest {
-  private MockClock clock;
-  @Mock private TransformWatermarks watermarks;
-
-  private TimerUpdateBuilder timerUpdateBuilder;
-
-  private DirectTimerInternals internals;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    clock = MockClock.fromInstant(new Instant(0));
-
-    timerUpdateBuilder = TimerUpdate.builder(StructuralKey.of(1234, VarIntCoder.of()));
-
-    internals = DirectTimerInternals.create(clock, watermarks, timerUpdateBuilder);
-  }
-
-  @Test
-  public void setTimerAddsToBuilder() {
-    TimerData eventTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
-    TimerData processingTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
-    TimerData synchronizedProcessingTimer =
-        TimerData.of(
-            StateNamespaces.global(),
-            new Instant(98745632189L),
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    internals.setTimer(eventTimer);
-    internals.setTimer(processingTimer);
-    internals.setTimer(synchronizedProcessingTimer);
-
-    assertThat(
-        internals.getTimerUpdate().getSetTimers(),
-        containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
-  }
-
-  @Test
-  public void deleteTimerDeletesOnBuilder() {
-    TimerData eventTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
-    TimerData processingTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
-    TimerData synchronizedProcessingTimer =
-        TimerData.of(
-            StateNamespaces.global(),
-            new Instant(98745632189L),
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    internals.deleteTimer(eventTimer);
-    internals.deleteTimer(processingTimer);
-    internals.deleteTimer(synchronizedProcessingTimer);
-
-    assertThat(
-        internals.getTimerUpdate().getDeletedTimers(),
-        containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
-  }
-
-  @Test
-  public void getProcessingTimeIsClockNow() {
-    assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
-    Instant oldProcessingTime = internals.currentProcessingTime();
-
-    clock.advance(Duration.standardHours(12));
-
-    assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
-    assertThat(
-        internals.currentProcessingTime(),
-        equalTo(oldProcessingTime.plus(Duration.standardHours(12))));
-  }
-
-  @Test
-  public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() {
-    when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L));
-    assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L)));
-  }
-
-  @Test
-  public void getInputWatermarkTimeUsesWatermarkTime() {
-    when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L));
-    assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L)));
-  }
-
-  @Test
-  public void getOutputWatermarkTimeUsesWatermarkTime() {
-    when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L));
-    assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L)));
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTransformExecutorTest.java
deleted file mode 100644
index 0cae265..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTransformExecutorTest.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DirectTransformExecutor}. */
-@RunWith(JUnit4.class)
-public class DirectTransformExecutorTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  private final PCollectionNode created =
-      PipelineNode.pCollection(
-          "created", PCollection.newBuilder().setUniqueName("created").build());
-
-  private final PTransformNode createdProducer =
-      PipelineNode.pTransform(
-          "create",
-          PTransform.newBuilder().putOutputs("created", "created").setUniqueName("create").build());
-  private final PTransformNode downstreamProducer =
-      PipelineNode.pTransform(
-          "downstream",
-          PTransform.newBuilder().putInputs("input", "created").setUniqueName("create").build());
-
-  private CountDownLatch evaluatorCompleted;
-
-  private RegisteringCompletionCallback completionCallback;
-  private TransformExecutorService transformEvaluationState;
-  private BundleFactory bundleFactory;
-  @Mock private DirectMetrics metrics;
-  @Mock private EvaluationContext evaluationContext;
-  @Mock private TransformEvaluatorRegistry registry;
-
-  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-
-    bundleFactory = ImmutableListBundleFactory.create();
-
-    transformEvaluationState =
-        TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService());
-
-    evaluatorCompleted = new CountDownLatch(1);
-    completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
-
-    PipelineNode.pCollection(
-        "created", RunnerApi.PCollection.newBuilder().setUniqueName("created").build());
-
-    when(evaluationContext.getMetrics()).thenReturn(metrics);
-  }
-
-  @Test
-  public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
-    final TransformResult<Object> result = StepTransformResult.withoutHold(createdProducer).build();
-    final AtomicBoolean finishCalled = new AtomicBoolean(false);
-    TransformEvaluator<Object> evaluator =
-        new TransformEvaluator<Object>() {
-          @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {
-            throw new IllegalArgumentException("Shouldn't be called");
-          }
-
-          @Override
-          public TransformResult<Object> finishBundle() throws Exception {
-            finishCalled.set(true);
-            return result;
-          }
-        };
-
-    when(registry.forApplication(createdProducer, null)).thenReturn(evaluator);
-
-    DirectTransformExecutor<Object> executor =
-        new DirectTransformExecutor<>(
-            evaluationContext,
-            registry,
-            null,
-            createdProducer,
-            completionCallback,
-            transformEvaluationState);
-    executor.run();
-
-    assertThat(finishCalled.get(), is(true));
-    assertThat(completionCallback.handledResult, Matchers.equalTo(result));
-    assertThat(completionCallback.handledException, is(nullValue()));
-  }
-
-  @Test
-  public void nullTransformEvaluatorTerminates() throws Exception {
-    when(registry.forApplication(createdProducer, null)).thenReturn(null);
-
-    DirectTransformExecutor<Object> executor =
-        new DirectTransformExecutor<>(
-            evaluationContext,
-            registry,
-            null,
-            createdProducer,
-            completionCallback,
-            transformEvaluationState);
-    executor.run();
-
-    assertThat(completionCallback.handledResult, is(nullValue()));
-    assertThat(completionCallback.handledEmpty, equalTo(true));
-    assertThat(completionCallback.handledException, is(nullValue()));
-  }
-
-  @Test
-  public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
-    final TransformResult<String> result =
-        StepTransformResult.<String>withoutHold(downstreamProducer).build();
-    final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
-    TransformEvaluator<String> evaluator =
-        new TransformEvaluator<String>() {
-          @Override
-          public void processElement(WindowedValue<String> element) throws Exception {
-            elementsProcessed.add(element);
-          }
-
-          @Override
-          public TransformResult<String> finishBundle() throws Exception {
-            return result;
-          }
-        };
-
-    WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
-    WindowedValue<String> spam = WindowedValue.valueInGlobalWindow("spam");
-    WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
-    CommittedBundle<String> inputBundle =
-        bundleFactory
-            .<String>createBundle(created)
-            .add(foo)
-            .add(spam)
-            .add(third)
-            .commit(Instant.now());
-    when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
-
-    DirectTransformExecutor<String> executor =
-        new DirectTransformExecutor<>(
-            evaluationContext,
-            registry,
-            inputBundle,
-            downstreamProducer,
-            completionCallback,
-            transformEvaluationState);
-
-    Future<?> future = Executors.newSingleThreadExecutor().submit(executor);
-
-    evaluatorCompleted.await();
-    future.get();
-
-    assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo));
-    assertThat(completionCallback.handledResult, Matchers.equalTo(result));
-    assertThat(completionCallback.handledException, is(nullValue()));
-  }
-
-  @Test
-  @SuppressWarnings("FutureReturnValueIgnored") // expected exception checked via completionCallback
-  public void processElementThrowsExceptionCallsback() throws Exception {
-    final TransformResult<String> result =
-        StepTransformResult.<String>withoutHold(downstreamProducer).build();
-    final Exception exception = new Exception();
-    TransformEvaluator<String> evaluator =
-        new TransformEvaluator<String>() {
-          @Override
-          public void processElement(WindowedValue<String> element) throws Exception {
-            throw exception;
-          }
-
-          @Override
-          public TransformResult<String> finishBundle() throws Exception {
-            return result;
-          }
-        };
-
-    WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
-    CommittedBundle<String> inputBundle =
-        bundleFactory.<String>createBundle(created).add(foo).commit(Instant.now());
-    when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
-
-    DirectTransformExecutor<String> executor =
-        new DirectTransformExecutor<>(
-            evaluationContext,
-            registry,
-            inputBundle,
-            downstreamProducer,
-            completionCallback,
-            transformEvaluationState);
-    Executors.newSingleThreadExecutor().submit(executor);
-
-    evaluatorCompleted.await();
-
-    assertThat(completionCallback.handledResult, is(nullValue()));
-    assertThat(completionCallback.handledException, Matchers.<Throwable>equalTo(exception));
-  }
-
-  @Test
-  @SuppressWarnings("FutureReturnValueIgnored") // expected exception checked via completionCallback
-  public void finishBundleThrowsExceptionCallsback() throws Exception {
-    final Exception exception = new Exception();
-    TransformEvaluator<String> evaluator =
-        new TransformEvaluator<String>() {
-          @Override
-          public void processElement(WindowedValue<String> element) throws Exception {}
-
-          @Override
-          public TransformResult<String> finishBundle() throws Exception {
-            throw exception;
-          }
-        };
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.<String>createBundle(created).commit(Instant.now());
-    when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
-
-    DirectTransformExecutor<String> executor =
-        new DirectTransformExecutor<>(
-            evaluationContext,
-            registry,
-            inputBundle,
-            downstreamProducer,
-            completionCallback,
-            transformEvaluationState);
-    Executors.newSingleThreadExecutor().submit(executor);
-
-    evaluatorCompleted.await();
-
-    assertThat(completionCallback.handledResult, is(nullValue()));
-    assertThat(completionCallback.handledException, Matchers.<Throwable>equalTo(exception));
-  }
-
-  private static class RegisteringCompletionCallback implements CompletionCallback {
-    private TransformResult<?> handledResult = null;
-    private boolean handledEmpty = false;
-    private Exception handledException = null;
-    private final CountDownLatch onMethod;
-
-    private RegisteringCompletionCallback(CountDownLatch onMethod) {
-      this.onMethod = onMethod;
-    }
-
-    @Override
-    public CommittedResult handleResult(CommittedBundle<?> inputBundle, TransformResult<?> result) {
-      handledResult = result;
-      onMethod.countDown();
-      @SuppressWarnings("rawtypes")
-      Iterable unprocessedElements =
-          result.getUnprocessedElements() == null
-              ? Collections.emptyList()
-              : result.getUnprocessedElements();
-
-      Optional<? extends CommittedBundle<?>> unprocessedBundle;
-      if (inputBundle == null || Iterables.isEmpty(unprocessedElements)) {
-        unprocessedBundle = Optional.absent();
-      } else {
-        unprocessedBundle =
-            Optional.<CommittedBundle<?>>of(inputBundle.withElements(unprocessedElements));
-      }
-      return CommittedResult.create(
-          result, unprocessedBundle, Collections.emptyList(), EnumSet.noneOf(OutputType.class));
-    }
-
-    @Override
-    public void handleEmpty(PTransformNode transform) {
-      handledEmpty = true;
-      onMethod.countDown();
-    }
-
-    @Override
-    public void handleException(CommittedBundle<?> inputBundle, Exception e) {
-      handledException = e;
-      onMethod.countDown();
-    }
-
-    @Override
-    public void handleError(Error err) {
-      throw err;
-    }
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/EvaluationContextTest.java
deleted file mode 100644
index 46df904..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/EvaluationContextTest.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import java.util.Collection;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableSet;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link EvaluationContext}. */
-@RunWith(JUnit4.class)
-public class EvaluationContextTest {
-  private EvaluationContext context;
-
-  private PCollectionNode created;
-  private PCollectionNode downstream;
-
-  private ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
-  private PTransformNode createdProducer;
-  private PTransformNode downstreamProducer;
-  private PTransformNode unboundedProducer;
-
-  @Before
-  public void setup() {
-    ExecutableGraphBuilder graphBuilder =
-        ExecutableGraphBuilder.create()
-            .addTransform("create", null, "created")
-            .addTransform("downstream", "created", "downstream.out")
-            .addTransform("unbounded", null, "unbounded.out");
-
-    graph = graphBuilder.toGraph();
-    created = graphBuilder.collectionNode("created");
-    downstream = graphBuilder.collectionNode("downstream.out");
-    createdProducer = graphBuilder.transformNode("create");
-    downstreamProducer = graphBuilder.transformNode("downstream");
-    unboundedProducer = graphBuilder.transformNode("unbounded");
-
-    BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-    context = EvaluationContext.create(Instant::new, bundleFactory, graph, ImmutableSet.of());
-  }
-
-  @Test
-  public void getExecutionContextSameStepSameKeyState() {
-    StepStateAndTimers<String> fooContext =
-        context.getStateAndTimers(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of()));
-
-    StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    fooContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
-    context.handleResult(
-        ImmutableListBundleFactory.create()
-            .createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), created)
-            .commit(Instant.now()),
-        ImmutableList.of(),
-        StepTransformResult.withoutHold(createdProducer)
-            .withState(fooContext.stateInternals().commit())
-            .build());
-
-    StepStateAndTimers secondFooContext =
-        context.getStateAndTimers(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of()));
-    assertThat(
-        secondFooContext.stateInternals().state(StateNamespaces.global(), intBag).read(),
-        contains(1));
-  }
-
-  @Test
-  public void getExecutionContextDifferentKeysIndependentState() {
-    StepStateAndTimers fooContext =
-        context.getStateAndTimers(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of()));
-
-    StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    fooContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
-    StepStateAndTimers barContext =
-        context.getStateAndTimers(createdProducer, StructuralKey.of("bar", StringUtf8Coder.of()));
-    assertThat(barContext, not(equalTo(fooContext)));
-    assertThat(
-        barContext.stateInternals().state(StateNamespaces.global(), intBag).read(),
-        emptyIterable());
-  }
-
-  @Test
-  public void getExecutionContextDifferentStepsIndependentState() {
-    StructuralKey<?> myKey = StructuralKey.of("foo", StringUtf8Coder.of());
-    StepStateAndTimers fooContext = context.getStateAndTimers(createdProducer, myKey);
-
-    StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    fooContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
-    StepStateAndTimers barContext = context.getStateAndTimers(downstreamProducer, myKey);
-    assertThat(
-        barContext.stateInternals().state(StateNamespaces.global(), intBag).read(),
-        emptyIterable());
-  }
-
-  @Test
-  public void handleResultStoresState() {
-    StructuralKey<?> myKey = StructuralKey.of("foo".getBytes(UTF_8), ByteArrayCoder.of());
-    StepStateAndTimers fooContext = context.getStateAndTimers(downstreamProducer, myKey);
-
-    StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    CopyOnAccessInMemoryStateInternals<?> state = fooContext.stateInternals();
-    BagState<Integer> bag = state.state(StateNamespaces.global(), intBag);
-    bag.add(1);
-    bag.add(2);
-    bag.add(4);
-
-    TransformResult<?> stateResult =
-        StepTransformResult.withoutHold(downstreamProducer).withState(state).build();
-
-    context.handleResult(
-        context.createKeyedBundle(myKey, created).commit(Instant.now()),
-        ImmutableList.of(),
-        stateResult);
-
-    StepStateAndTimers afterResultContext = context.getStateAndTimers(downstreamProducer, myKey);
-
-    CopyOnAccessInMemoryStateInternals<?> afterResultState = afterResultContext.stateInternals();
-    assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4));
-  }
-
-  @Test
-  public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() throws Exception {
-    final CountDownLatch callLatch = new CountDownLatch(1);
-    Runnable callback = callLatch::countDown;
-
-    // Should call back after the end of the global window
-    context.scheduleAfterOutputWouldBeProduced(
-        downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
-
-    TransformResult<?> result =
-        StepTransformResult.withHold(createdProducer, new Instant(0)).build();
-
-    context.handleResult(null, ImmutableList.of(), result);
-    // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit
-    // will likely be flaky if this logic is broken
-    assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false));
-
-    TransformResult<?> finishedResult = StepTransformResult.withoutHold(createdProducer).build();
-    context.handleResult(null, ImmutableList.of(), finishedResult);
-    context.forceRefresh();
-    // Obtain the value via blocking call
-    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
-  }
-
-  @Test
-  public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
-    TransformResult<?> finishedResult = StepTransformResult.withoutHold(createdProducer).build();
-    context.handleResult(null, ImmutableList.of(), finishedResult);
-
-    final CountDownLatch callLatch = new CountDownLatch(1);
-    context.extractFiredTimers();
-    Runnable callback = callLatch::countDown;
-    context.scheduleAfterOutputWouldBeProduced(
-        downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
-    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
-  }
-
-  @Test
-  public void extractFiredTimersExtractsTimers() {
-    TransformResult<?> holdResult =
-        StepTransformResult.withHold(createdProducer, new Instant(0)).build();
-    context.handleResult(null, ImmutableList.of(), holdResult);
-
-    StructuralKey<?> key = StructuralKey.of("foo".length(), VarIntCoder.of());
-    TimerData toFire =
-        TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
-    TransformResult<?> timerResult =
-        StepTransformResult.withoutHold(downstreamProducer)
-            .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null))
-            .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build())
-            .build();
-
-    // haven't added any timers, must be empty
-    assertThat(context.extractFiredTimers(), emptyIterable());
-    context.handleResult(
-        context.createKeyedBundle(key, created).commit(Instant.now()),
-        ImmutableList.of(),
-        timerResult);
-
-    // timer hasn't fired
-    assertThat(context.extractFiredTimers(), emptyIterable());
-
-    TransformResult<?> advanceResult = StepTransformResult.withoutHold(createdProducer).build();
-    // Should cause the downstream timer to fire
-    context.handleResult(null, ImmutableList.of(), advanceResult);
-
-    Collection<FiredTimers<PTransformNode>> fired = context.extractFiredTimers();
-    assertThat(Iterables.getOnlyElement(fired).getKey(), Matchers.equalTo(key));
-
-    FiredTimers<PTransformNode> firedForKey = Iterables.getOnlyElement(fired);
-    // Contains exclusively the fired timer
-    assertThat(firedForKey.getTimers(), contains(toFire));
-
-    // Don't reextract timers
-    assertThat(context.extractFiredTimers(), emptyIterable());
-  }
-
-  @Test
-  public void createKeyedBundleKeyed() {
-    StructuralKey<String> key = StructuralKey.of("foo", StringUtf8Coder.of());
-    CommittedBundle<KV<String, Integer>> keyedBundle =
-        context
-            .<String, KV<String, Integer>>createKeyedBundle(key, downstream)
-            .commit(Instant.now());
-    assertThat(keyedBundle.getKey(), Matchers.equalTo(key));
-  }
-
-  @Test
-  public void isDoneWithUnboundedPCollection() {
-    assertThat(context.isDone(unboundedProducer), is(false));
-
-    context.handleResult(
-        null, ImmutableList.of(), StepTransformResult.withoutHold(unboundedProducer).build());
-    context.extractFiredTimers();
-    assertThat(context.isDone(unboundedProducer), is(true));
-  }
-
-  @Test
-  public void isDoneWithPartiallyDone() {
-    assertThat(context.isDone(), is(false));
-
-    UncommittedBundle<Integer> rootBundle = context.createBundle(created);
-    rootBundle.add(WindowedValue.valueInGlobalWindow(1));
-    CommittedResult handleResult =
-        context.handleResult(
-            null,
-            ImmutableList.of(),
-            StepTransformResult.<Integer>withoutHold(createdProducer)
-                .addOutput(rootBundle)
-                .build());
-    @SuppressWarnings("unchecked")
-    CommittedBundle<Integer> committedBundle =
-        (CommittedBundle<Integer>) Iterables.getOnlyElement(handleResult.getOutputs());
-    context.handleResult(
-        null, ImmutableList.of(), StepTransformResult.withoutHold(unboundedProducer).build());
-    assertThat(context.isDone(), is(false));
-
-    for (PTransformNode consumers : graph.getPerElementConsumers(created)) {
-      context.handleResult(
-          committedBundle, ImmutableList.of(), StepTransformResult.withoutHold(consumers).build());
-    }
-    context.extractFiredTimers();
-    assertThat(context.isDone(), is(true));
-  }
-
-  private static class TestBoundedWindow extends BoundedWindow {
-    private final Instant ts;
-
-    public TestBoundedWindow(Instant ts) {
-      this.ts = ts;
-    }
-
-    @Override
-    public Instant maxTimestamp() {
-      return ts;
-    }
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/ExecutableGraphBuilder.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/ExecutableGraphBuilder.java
deleted file mode 100644
index f0f5b70..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/ExecutableGraphBuilder.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-
-/**
- * A builder of simple {@link ExecutableGraph ExecutableGraphs} suitable for use in the portable
- * direct runner, to reduce verbosity of creating a graph with no payloads of any meaning.
- */
-public class ExecutableGraphBuilder {
-  private final RunnerApi.Components.Builder components;
-
-  private ExecutableGraphBuilder() {
-    components = Components.newBuilder();
-  }
-
-  public static ExecutableGraphBuilder create() {
-    return new ExecutableGraphBuilder();
-  }
-
-  public ExecutableGraphBuilder addTransform(
-      String name, @Nullable String input, String... outputs) {
-    PTransform.Builder pt = PTransform.newBuilder().setUniqueName(name);
-    if (input != null) {
-      pt = pt.putInputs("input", input);
-      addPCollection(input);
-    }
-    for (String output : outputs) {
-      pt = pt.putOutputs(output, output);
-      addPCollection(output);
-    }
-    components.putTransforms(name, pt.build());
-    return this;
-  }
-
-  private ExecutableGraphBuilder addPCollection(String name) {
-    components.putPcollections(name, PCollection.newBuilder().setUniqueName(name).build());
-    return this;
-  }
-
-  public PTransformNode transformNode(String name) {
-    return PipelineNode.pTransform(name, components.getTransformsOrThrow(name));
-  }
-
-  public PCollectionNode collectionNode(String name) {
-    return PipelineNode.pCollection(name, components.getPcollectionsOrThrow(name));
-  }
-
-  public ExecutableGraph<PTransformNode, PCollectionNode> toGraph() {
-    return PortableGraph.forPipeline(
-        Pipeline.newBuilder()
-            .setComponents(components)
-            .addAllRootTransformIds(components.getTransformsMap().keySet())
-            .build());
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactoryTest.java
deleted file mode 100644
index e183895..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * 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 org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link FlattenEvaluatorFactory}. */
-@RunWith(JUnit4.class)
-public class FlattenEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-
-  @Test
-  public void testFlattenInMemoryEvaluator() throws Exception {
-    PCollectionNode left =
-        PipelineNode.pCollection("left", PCollection.newBuilder().setUniqueName("left").build());
-    PCollectionNode right =
-        PipelineNode.pCollection("right", PCollection.newBuilder().setUniqueName("right").build());
-    // Include a root node for a sane-looking graph
-    PTransformNode source =
-        PipelineNode.pTransform(
-            "source",
-            PTransform.newBuilder()
-                .putOutputs("left", left.getId())
-                .putOutputs("right", right.getId())
-                .build());
-
-    PCollectionNode flattened =
-        PipelineNode.pCollection("flat", PCollection.newBuilder().setUniqueName("flat").build());
-    PTransformNode flatten =
-        PipelineNode.pTransform(
-            "flatten",
-            PTransform.newBuilder()
-                .setUniqueName("flatten")
-                .putInputs("left", left.getId())
-                .putInputs("right", right.getId())
-                .putOutputs("out", flattened.getId())
-                .setSpec(
-                    FunctionSpec.newBuilder().setUrn(PTransformTranslation.FLATTEN_TRANSFORM_URN))
-                .build());
-
-    PortableGraph graph =
-        PortableGraph.forPipeline(
-            RunnerApi.Pipeline.newBuilder()
-                .addRootTransformIds(source.getId())
-                .addRootTransformIds(flatten.getId())
-                .setComponents(
-                    RunnerApi.Components.newBuilder()
-                        .putTransforms(source.getId(), source.getTransform())
-                        .putPcollections(left.getId(), left.getPCollection())
-                        .putPcollections(right.getId(), right.getPCollection())
-                        .putTransforms(flatten.getId(), flatten.getTransform())
-                        .putPcollections(flattened.getId(), flattened.getPCollection()))
-                .build());
-
-    CommittedBundle<Integer> leftBundle =
-        bundleFactory.<Integer>createBundle(left).commit(Instant.now());
-    CommittedBundle<Integer> rightBundle =
-        bundleFactory.<Integer>createBundle(right).commit(Instant.now());
-
-    FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory(graph, bundleFactory);
-    TransformEvaluator<Integer> leftSideEvaluator = factory.forApplication(flatten, leftBundle);
-    TransformEvaluator<Integer> rightSideEvaluator = factory.forApplication(flatten, rightBundle);
-
-    leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1));
-    rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1));
-    leftSideEvaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)));
-    leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING));
-    rightSideEvaluator.processElement(
-        WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING));
-    rightSideEvaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
-
-    TransformResult<Integer> rightSideResult = rightSideEvaluator.finishBundle();
-    TransformResult<Integer> leftSideResult = leftSideEvaluator.finishBundle();
-
-    assertThat(
-        getOnlyElement(leftSideResult.getOutputBundles()).commit(Instant.now()),
-        containsInAnyOrder(
-            WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)),
-            WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING),
-            WindowedValue.valueInGlobalWindow(1)));
-    assertThat(
-        getOnlyElement(rightSideResult.getOutputBundles()).commit(Instant.now()),
-        containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)),
-            WindowedValue.valueInGlobalWindow(-1)));
-  }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
deleted file mode 100644
index 10db613..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * 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
... 3153 lines suppressed ...