You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2018/10/08 19:04:32 UTC

[beam] branch master updated: [BEAM-5634] Bring dataflow java worker code into beam (#6561)

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

kenn 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 1c0a0bf  [BEAM-5634] Bring dataflow java worker code into beam (#6561)
1c0a0bf is described below

commit 1c0a0bfba4fb45c78b0ade3284d641c67e4688f3
Author: Boyuan Zhang <36...@users.noreply.github.com>
AuthorDate: Mon Oct 8 12:04:24 2018 -0700

    [BEAM-5634] Bring dataflow java worker code into beam (#6561)
    
    * Donating dataflow java worker code into apache beam
    * Apply Java Spotless
    * Migrate dataflow java worker build into beam project build
---
 .../google-cloud-dataflow-java/worker/build.gradle |  215 ++
 .../dataflow/harness/util/ThrowingBiConsumer.java  |   32 +
 .../dataflow/harness/util/ThrowingBiFunction.java  |   32 +
 .../dataflow/harness/util/ThrowingConsumer.java    |   32 +
 .../dataflow/harness/util/ThrowingFunction.java    |   32 +
 .../dataflow/harness/util/ThrowingRunnable.java    |   30 +
 .../dataflow/harness/util/package-info.java        |   24 +
 .../dataflow/worker/ApplianceShuffleCounters.java  |   79 +
 .../worker/ApplianceShuffleEntryReader.java        |   69 +
 .../dataflow/worker/ApplianceShuffleReader.java    |   98 +
 .../dataflow/worker/ApplianceShuffleWriter.java    |   72 +
 .../worker/AssignWindowsParDoFnFactory.java        |  138 ++
 .../runners/dataflow/worker/AvroByteReader.java    |  179 ++
 .../dataflow/worker/AvroByteReaderFactory.java     |   67 +
 .../beam/runners/dataflow/worker/AvroByteSink.java |   87 +
 .../dataflow/worker/AvroByteSinkFactory.java       |   58 +
 .../dataflow/worker/BatchDataflowWorker.java       |  430 ++++
 .../dataflow/worker/BatchModeExecutionContext.java |  517 ++++
 .../worker/BatchModeUngroupingParDoFn.java         |   83 +
 .../worker/BeamFnMapTaskExecutorFactory.java       |  653 +++++
 .../runners/dataflow/worker/ByteStringCoder.java   |   53 +
 .../worker/ChunkingShuffleBatchReader.java         |  103 +
 .../beam/runners/dataflow/worker/CombinePhase.java |   34 +
 .../dataflow/worker/CombineValuesFnFactory.java    |  338 +++
 .../beam/runners/dataflow/worker/ConcatReader.java |  272 +++
 .../dataflow/worker/ConcatReaderFactory.java       |  151 ++
 .../dataflow/worker/ContextActivationObserver.java |   60 +
 .../worker/ContextActivationObserverRegistry.java  |   84 +
 .../dataflow/worker/CounterShortIdCache.java       |  155 ++
 .../CreateIsmShardKeyAndSortKeyDoFnFactory.java    |  120 +
 .../runners/dataflow/worker/DataflowApiUtils.java  |   60 +
 .../worker/DataflowBatchWorkerHarness.java         |  174 ++
 .../worker/DataflowElementExecutionTracker.java    |  449 ++++
 .../dataflow/worker/DataflowExecutionContext.java  |  277 +++
 .../dataflow/worker/DataflowMapTaskExecutor.java   |   39 +
 .../worker/DataflowMapTaskExecutorFactory.java     |   53 +
 .../dataflow/worker/DataflowMetricsContainer.java  |   62 +
 .../dataflow/worker/DataflowOperationContext.java  |  300 +++
 .../dataflow/worker/DataflowOutputCounter.java     |  101 +
 .../worker/DataflowPortabilityPCollectionView.java |  177 ++
 .../dataflow/worker/DataflowProcessFnRunner.java   |  125 +
 .../dataflow/worker/DataflowRunnerHarness.java     |  194 ++
 .../worker/DataflowSideInputReadCounter.java       |  179 ++
 .../dataflow/worker/DataflowSystemMetrics.java     |   86 +
 .../dataflow/worker/DataflowWorkExecutor.java      |   32 +
 .../worker/DataflowWorkProgressUpdater.java        |  121 +
 .../dataflow/worker/DataflowWorkUnitClient.java    |  257 ++
 .../worker/DataflowWorkerHarnessHelper.java        |  122 +
 .../dataflow/worker/DefaultParDoFnFactory.java     |   85 +
 .../runners/dataflow/worker/DeltaCounterCell.java  |   84 +
 .../dataflow/worker/DeltaDistributionCell.java     |   75 +
 .../dataflow/worker/DoFnInstanceManager.java       |   47 +
 .../dataflow/worker/DoFnInstanceManagers.java      |  123 +
 .../runners/dataflow/worker/DoFnRunnerFactory.java |   47 +
 .../runners/dataflow/worker/ExecutionStateKey.java |   87 +
 .../dataflow/worker/ExecutionStateRegistry.java    |  115 +
 .../runners/dataflow/worker/ExperimentContext.java |   91 +
 ...FetchAndFilterStreamingSideInputsOperation.java |  272 +++
 .../beam/runners/dataflow/worker/Filepatterns.java |   82 +
 .../dataflow/worker/FnApiWindowMappingFn.java      |  330 +++
 .../runners/dataflow/worker/ForwardingParDoFn.java |   60 +
 .../dataflow/worker/GroupAlsoByWindowFn.java       |   40 +
 .../dataflow/worker/GroupAlsoByWindowFnRunner.java |  132 +
 .../worker/GroupAlsoByWindowParDoFnFactory.java    |  383 +++
 .../dataflow/worker/GroupAlsoByWindowsParDoFn.java |  236 ++
 .../dataflow/worker/GroupingShuffleReader.java     |  446 ++++
 .../worker/GroupingShuffleReaderFactory.java       |  105 +
 ...ingShuffleReaderWithFaultyBytesReadCounter.java |   72 +
 .../runners/dataflow/worker/InMemoryReader.java    |  187 ++
 .../dataflow/worker/InMemoryReaderFactory.java     |   66 +
 .../dataflow/worker/IntrinsicMapTaskExecutor.java  |   79 +
 .../worker/IntrinsicMapTaskExecutorFactory.java    |  403 +++
 .../beam/runners/dataflow/worker/IsmReader.java    |  140 ++
 .../runners/dataflow/worker/IsmReaderFactory.java  |  157 ++
 .../runners/dataflow/worker/IsmReaderImpl.java     | 1160 +++++++++
 .../dataflow/worker/IsmSideInputReader.java        | 1088 +++++++++
 .../beam/runners/dataflow/worker/IsmSink.java      |  307 +++
 .../runners/dataflow/worker/IsmSinkFactory.java    |  112 +
 .../dataflow/worker/KeyTokenInvalidException.java  |   37 +
 .../worker/LazilyInitializedSideInputReader.java   |   64 +
 .../worker/MetricTrackingWindmillServerStub.java   |  266 ++
 .../dataflow/worker/MetricsContainerRegistry.java  |   61 +
 ...nmentContextActivationObserverRegistration.java |   63 +
 .../worker/MetricsToCounterUpdateConverter.java    |  108 +
 .../worker/NoOpSourceOperationExecutor.java        |   66 +
 .../dataflow/worker/NoopSideInputReadCounter.java  |   34 +
 .../beam/runners/dataflow/worker/OrderedCode.java  |  647 +++++
 .../dataflow/worker/PCollectionViewWindow.java     |   66 +
 .../worker/PairWithConstantKeyDoFnFactory.java     |  104 +
 .../runners/dataflow/worker/ParDoFnFactory.java    |   48 +
 .../dataflow/worker/PartialGroupByKeyParDoFns.java |  376 +++
 .../dataflow/worker/PartitioningShuffleReader.java |  162 ++
 .../worker/PartitioningShuffleReaderFactory.java   |   81 +
 .../beam/runners/dataflow/worker/PubsubReader.java |  133 +
 .../beam/runners/dataflow/worker/PubsubSink.java   |  197 ++
 .../beam/runners/dataflow/worker/ReaderCache.java  |  141 ++
 .../runners/dataflow/worker/ReaderFactory.java     |   68 +
 .../runners/dataflow/worker/ReaderRegistry.java    |  130 +
 .../beam/runners/dataflow/worker/ReaderUtils.java  |   74 +
 .../ReifyTimestampAndWindowsParDoFnFactory.java    |   93 +
 ...HarnessCoderCloudObjectTranslatorRegistrar.java |  136 ++
 .../dataflow/worker/SdkHarnessRegistries.java      |  275 +++
 .../dataflow/worker/SdkHarnessRegistry.java        |   69 +
 .../runners/dataflow/worker/ShuffleLibrary.java    |   53 +
 .../runners/dataflow/worker/ShuffleReader.java     |   47 +
 .../beam/runners/dataflow/worker/ShuffleSink.java  |  351 +++
 .../dataflow/worker/ShuffleSinkFactory.java        |   73 +
 .../runners/dataflow/worker/ShuffleWriter.java     |   35 +
 .../dataflow/worker/SideInputReadCounter.java      |   28 +
 .../worker/SideInputTrackingIsmReader.java         |  108 +
 .../dataflow/worker/SimpleDoFnRunnerFactory.java   |   77 +
 .../runners/dataflow/worker/SimpleParDoFn.java     |  495 ++++
 .../beam/runners/dataflow/worker/SinkFactory.java  |   60 +
 .../beam/runners/dataflow/worker/SinkRegistry.java |  110 +
 .../dataflow/worker/SizeReportingSinkWrapper.java  |   79 +
 .../dataflow/worker/SourceOperationExecutor.java   |   31 +
 .../worker/SourceOperationExecutorFactory.java     |   61 +
 .../dataflow/worker/SourceTranslationUtils.java    |  160 ++
 .../worker/SplittableProcessFnFactory.java         |  192 ++
 .../beam/runners/dataflow/worker/StateFetcher.java |  268 ++
 .../dataflow/worker/StreamingDataflowWorker.java   | 2042 ++++++++++++++++
 .../StreamingGroupAlsoByWindowReshuffleFn.java     |   65 +
 .../StreamingGroupAlsoByWindowViaWindowSetFn.java  |  106 +
 .../worker/StreamingGroupAlsoByWindowsDoFns.java   |   57 +
 .../StreamingKeyedWorkItemSideInputDoFnRunner.java |  160 ++
 .../worker/StreamingModeExecutionContext.java      |  789 ++++++
 .../StreamingPCollectionViewWriterDoFnFactory.java |   77 +
 .../StreamingPCollectionViewWriterParDoFn.java     |   83 +
 .../worker/StreamingSideInputDoFnRunner.java       |   96 +
 .../dataflow/worker/StreamingSideInputFetcher.java |  361 +++
 .../worker/StreamingStepMetricsContainer.java      |  133 +
 .../worker/ToIsmRecordForMultimapDoFnFactory.java  |  155 ++
 .../dataflow/worker/UngroupedShuffleReader.java    |  129 +
 .../worker/UngroupedShuffleReaderFactory.java      |   74 +
 .../dataflow/worker/UngroupedWindmillReader.java   |  129 +
 .../dataflow/worker/UserParDoFnFactory.java        |  152 ++
 .../runners/dataflow/worker/ValuesDoFnFactory.java |   87 +
 .../beam/runners/dataflow/worker/Weighers.java     |   40 +
 .../dataflow/worker/WindmillKeyedWorkItem.java     |  218 ++
 .../dataflow/worker/WindmillNamespacePrefix.java   |   46 +
 .../worker/WindmillReaderIteratorBase.java         |   75 +
 .../beam/runners/dataflow/worker/WindmillSink.java |  206 ++
 .../dataflow/worker/WindmillStateCache.java        |  373 +++
 .../dataflow/worker/WindmillStateInternals.java    | 1021 ++++++++
 .../dataflow/worker/WindmillStateReader.java       |  722 ++++++
 .../runners/dataflow/worker/WindmillTimeUtils.java |   71 +
 .../dataflow/worker/WindmillTimerInternals.java    |  334 +++
 .../dataflow/worker/WindowingWindmillReader.java   |  144 ++
 .../dataflow/worker/WorkItemStatusClient.java      |  333 +++
 .../runners/dataflow/worker/WorkUnitClient.java    |   53 +
 .../WorkerCustomSourceOperationExecutor.java       |   98 +
 .../dataflow/worker/WorkerCustomSources.java       |  845 +++++++
 .../worker/WorkerPipelineOptionsFactory.java       |   82 +
 .../worker/WorkerUncaughtExceptionHandler.java     |   66 +
 .../runners/dataflow/worker/apiary/Apiary.java     |   46 +
 .../FixMultiOutputInfosOnParDoInstructions.java    |   66 +
 .../runners/dataflow/worker/counters/Counter.java  |  261 ++
 .../CounterBackedElementByteSizeObserver.java      |   38 +
 .../dataflow/worker/counters/CounterFactory.java   |  877 +++++++
 .../dataflow/worker/counters/CounterName.java      |  308 +++
 .../dataflow/worker/counters/CounterSet.java       |  152 ++
 .../counters/DataflowCounterUpdateExtractor.java   |  225 ++
 .../dataflow/worker/counters/NameContext.java      |   91 +
 .../dataflow/worker/fn/BeamFnControlService.java   |  100 +
 .../runners/dataflow/worker/fn/IdGenerator.java    |   35 +
 .../runners/dataflow/worker/fn/ServerFactory.java  |  230 ++
 .../dataflow/worker/fn/SocketAddressFactory.java   |   69 +
 .../worker/fn/control/BeamFnMapTaskExecutor.java   |  593 +++++
 .../control/RegisterAndProcessBundleOperation.java |  600 +++++
 .../worker/fn/data/BeamFnDataGrpcService.java      |  271 +++
 .../fn/data/RemoteGrpcPortReadOperation.java       |  117 +
 .../fn/data/RemoteGrpcPortWriteOperation.java      |  128 +
 .../dataflow/worker/fn/grpc/BeamFnService.java     |   33 +
 .../worker/fn/logging/BeamFnLoggingService.java    |  153 ++
 .../fn/stream/ServerStreamObserverFactory.java     |  101 +
 .../graph/CloneAmbiguousFlattensFunction.java      |  146 ++
 .../graph/CreateRegisterFnOperationFunction.java   |  299 +++
 .../graph/DeduceFlattenLocationsFunction.java      |  325 +++
 .../worker/graph/DeduceNodeLocationsFunction.java  |  124 +
 .../beam/runners/dataflow/worker/graph/Edges.java  |   97 +
 ...nsertFetchAndFilterStreamingSideInputNodes.java |  164 ++
 .../worker/graph/LengthPrefixUnknownCoders.java    |  346 +++
 .../worker/graph/MapTaskToNetworkFunction.java     |  161 ++
 .../runners/dataflow/worker/graph/Networks.java    |  250 ++
 .../beam/runners/dataflow/worker/graph/Nodes.java  |  331 +++
 .../worker/graph/RegisterNodeFunction.java         |  604 +++++
 .../graph/RemoveFlattenInstructionsFunction.java   |   81 +
 .../graph/ReplacePgbkWithPrecombineFunction.java   |   84 +
 .../logging/DataflowWorkerLoggingHandler.java      |  339 +++
 .../logging/DataflowWorkerLoggingInitializer.java  |  225 ++
 .../worker/logging/DataflowWorkerLoggingMDC.java   |   78 +
 .../JulHandlerPrintStreamAdapterFactory.java       |  155 ++
 .../options/StreamingDataflowWorkerOptions.java    |  230 ++
 .../runners/dataflow/worker/profiler/Profiler.java |   81 +
 .../dataflow/worker/profiler/ScopedProfiler.java   |  196 ++
 .../dataflow/worker/status/BaseStatusServlet.java  |   56 +
 .../dataflow/worker/status/DebugCapture.java       |  208 ++
 .../dataflow/worker/status/HealthzServlet.java     |   38 +
 .../dataflow/worker/status/HeapzServlet.java       |  103 +
 .../worker/status/LastExceptionDataProvider.java   |   44 +
 .../dataflow/worker/status/StatusDataProvider.java |   25 +
 .../dataflow/worker/status/StatuszServlet.java     |   79 +
 .../dataflow/worker/status/ThreadzServlet.java     |  135 +
 .../dataflow/worker/status/WorkerStatusPages.java  |  126 +
 .../dataflow/worker/status/package-info.java       |   23 +
 .../util/BatchGroupAlsoByWindowAndCombineFn.java   |  304 +++
 .../worker/util/BatchGroupAlsoByWindowFn.java      |   26 +
 .../util/BatchGroupAlsoByWindowReshuffleFn.java    |   64 +
 .../util/BatchGroupAlsoByWindowViaIteratorsFn.java |  281 +++
 .../BatchGroupAlsoByWindowViaOutputBufferFn.java   |  130 +
 .../worker/util/BatchGroupAlsoByWindowsDoFns.java  |   62 +
 .../dataflow/worker/util/BoundedQueueExecutor.java |   77 +
 .../dataflow/worker/util/CloudSourceUtils.java     |   48 +
 .../dataflow/worker/util/FluentBackoff.java        |  237 ++
 .../dataflow/worker/util/MemoryMonitor.java        |  621 +++++
 .../dataflow/worker/util/ScalableBloomFilter.java  |  267 ++
 .../worker/util/StreamingGroupAlsoByWindowFn.java  |   26 +
 .../dataflow/worker/util/TimerOrElement.java       |  121 +
 .../dataflow/worker/util/ValueInEmptyWindows.java  |   94 +
 .../dataflow/worker/util/WorkerPropertyNames.java  |   47 +
 .../worker/util/common/ForwardingReiterator.java   |   88 +
 .../worker/util/common/TaggedReiteratorList.java   |  168 ++
 .../common/worker/BatchingShuffleEntryReader.java  |  142 ++
 .../common/worker/ByteArrayShufflePosition.java    |  109 +
 .../common/worker/CachingShuffleBatchReader.java   |  106 +
 .../worker/util/common/worker/ElementCounter.java  |   28 +
 .../common/worker/ElementExecutionTracker.java     |   58 +
 .../util/common/worker/ExecutionStateSampler.java  |  175 ++
 .../util/common/worker/ExecutionStateTracker.java  |  302 +++
 .../util/common/worker/FlattenOperation.java       |   51 +
 .../worker/GroupingShuffleEntryIterator.java       |  336 +++
 .../common/worker/GroupingShuffleRangeTracker.java |  196 ++
 .../worker/util/common/worker/GroupingTable.java   |   29 +
 .../worker/util/common/worker/GroupingTables.java  |  526 ++++
 .../worker/util/common/worker/JvmRuntime.java      |   38 +
 .../common/worker/KeyGroupedShuffleEntries.java    |   35 +
 .../worker/util/common/worker/MapTaskExecutor.java |  173 ++
 .../worker/util/common/worker/NativeReader.java    |  265 ++
 .../worker/util/common/worker/Operation.java       |  156 ++
 .../util/common/worker/OperationContext.java       |   80 +
 .../common/worker/OutputObjectAndByteCounter.java  |  176 ++
 .../worker/util/common/worker/OutputReceiver.java  |   72 +
 .../worker/util/common/worker/ParDoFn.java         |   37 +
 .../worker/util/common/worker/ParDoOperation.java  |   79 +
 .../worker/util/common/worker/ProgressTracker.java |   38 +
 .../util/common/worker/ProgressTrackerGroup.java   |   71 +
 .../common/worker/ProgressTrackingReiterator.java  |   55 +
 .../worker/util/common/worker/ReadOperation.java   |  435 ++++
 .../worker/util/common/worker/Receiver.java        |   25 +
 .../util/common/worker/ReceivingOperation.java     |   36 +
 .../util/common/worker/ShuffleBatchReader.java     |   55 +
 .../worker/util/common/worker/ShuffleEntry.java    |  111 +
 .../util/common/worker/ShuffleEntryReader.java     |   45 +
 .../worker/util/common/worker/ShufflePosition.java |   22 +
 .../util/common/worker/ShuffleReadCounter.java     |   91 +
 .../common/worker/ShuffleReadCounterFactory.java   |   36 +
 .../worker/SimplePartialGroupByKeyParDoFn.java     |   50 +
 .../dataflow/worker/util/common/worker/Sink.java   |   60 +
 .../worker/util/common/worker/WorkExecutor.java    |   84 +
 .../util/common/worker/WorkProgressUpdater.java    |  371 +++
 .../worker/util/common/worker/WriteOperation.java  |  149 ++
 .../worker/windmill/DirectStreamObserver.java      |   74 +
 .../windmill/ForwardingClientResponseObserver.java |   67 +
 .../worker/windmill/GrpcWindmillServer.java        | 1407 +++++++++++
 .../worker/windmill/StreamObserverFactory.java     |   56 +
 .../dataflow/worker/windmill/WindmillServer.java   |   53 +
 .../worker/windmill/WindmillServerBase.java        |  142 ++
 .../worker/windmill/WindmillServerStub.java        |  237 ++
 .../worker/src/main/resources/LICENSE              |  202 ++
 .../worker/src/main/resources/NOTICES              |  641 +++++
 .../dataflow/harness/test/TestExecutors.java       |   85 +
 .../dataflow/harness/test/TestExecutorsTest.java   |  161 ++
 .../runners/dataflow/harness/test/TestStreams.java |  149 ++
 .../dataflow/harness/test/TestStreamsTest.java     |   82 +
 .../worker/ApplianceShuffleCountersTest.java       |  150 ++
 .../dataflow/worker/AvroByteReaderFactoryTest.java |  101 +
 .../dataflow/worker/AvroByteReaderTest.java        |  365 +++
 .../dataflow/worker/AvroByteSinkFactoryTest.java   |   69 +
 .../runners/dataflow/worker/AvroByteSinkTest.java  |   88 +
 .../dataflow/worker/BatchDataflowWorkerTest.java   |  192 ++
 .../worker/BatchModeExecutionContextTest.java      |  314 +++
 .../worker/CombineValuesFnFactoryTest.java         |  361 +++
 .../dataflow/worker/ConcatReaderFactoryTest.java   |  122 +
 .../runners/dataflow/worker/ConcatReaderTest.java  |  501 ++++
 .../ContextActivationObserverRegistryTest.java     |   76 +
 .../dataflow/worker/CounterShortIdCacheTest.java   |  251 ++
 ...CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java |   89 +
 .../worker/DataflowBatchWorkerHarnessTest.java     |  201 ++
 .../DataflowElementExecutionTrackerTest.java       |  249 ++
 .../worker/DataflowExecutionContextTest.java       |  114 +
 .../runners/dataflow/worker/DataflowMatchers.java  |   56 +
 .../worker/DataflowOperationContextTest.java       |  280 +++
 .../DataflowPortabilityPCollectionViewTest.java    |   73 +
 .../worker/DataflowSideInputReadCounterTest.java   |  218 ++
 .../worker/DataflowWorkProgressUpdaterTest.java    |  253 ++
 .../worker/DataflowWorkUnitClientTest.java         |  226 ++
 .../worker/DataflowWorkerHarnessHelperTest.java    |   86 +
 .../dataflow/worker/DefaultParDoFnFactoryTest.java |  152 ++
 .../dataflow/worker/DeltaCounterCellTest.java      |  107 +
 .../dataflow/worker/DoFnInstanceManagersTest.java  |  162 ++
 .../dataflow/worker/ExperimentContextTest.java     |   75 +
 .../dataflow/worker/FakeWindmillServer.java        |  255 ++
 .../runners/dataflow/worker/FilepatternsTest.java  |   76 +
 .../dataflow/worker/FnApiWindowMappingFnTest.java  |  174 ++
 .../GroupAlsoByWindowParDoFnFactoryTest.java       |  104 +
 .../dataflow/worker/GroupingShuffleReaderTest.java |  981 ++++++++
 .../dataflow/worker/InMemoryReaderFactoryTest.java |   97 +
 .../dataflow/worker/InMemoryReaderTest.java        |  301 +++
 .../IntrinsicMapTaskExecutorFactoryTest.java       |  749 ++++++
 .../worker/IntrinsicMapTaskExecutorTest.java       |  503 ++++
 .../runners/dataflow/worker/IsmFormatTest.java     |  323 +++
 .../dataflow/worker/IsmReaderFactoryTest.java      |  165 ++
 .../runners/dataflow/worker/IsmReaderTest.java     |  797 ++++++
 .../dataflow/worker/IsmSideInputReaderTest.java    | 1813 ++++++++++++++
 .../beam/runners/dataflow/worker/IsmSinkTest.java  |  179 ++
 .../worker/KeyTokenInvalidExceptionTest.java       |   40 +
 .../LazilyInitializedSideInputReaderTest.java      |   87 +
 .../runners/dataflow/worker/LogRecordMatcher.java  |   82 +
 .../dataflow/worker/LogRecordMatcherTest.java      |   98 +
 .../beam/runners/dataflow/worker/LogSaver.java     |   46 +
 .../beam/runners/dataflow/worker/LogSaverTest.java |   68 +
 ...tContextActivationObserverRegistrationTest.java |   45 +
 .../dataflow/worker/NameContextsForTests.java      |   34 +
 .../worker/NoOpSourceOperationExecutorTest.java    |   66 +
 .../runners/dataflow/worker/OrderedCodeTest.java   |  490 ++++
 .../worker/PairWithConstantKeyDoFnFactoryTest.java |   68 +
 .../worker/PartialGroupByKeyParDoFnsTest.java      |  461 ++++
 .../worker/PartitioningShuffleReaderTest.java      |  150 ++
 .../runners/dataflow/worker/PubsubReaderTest.java  |  115 +
 .../runners/dataflow/worker/PubsubSinkTest.java    |  121 +
 .../runners/dataflow/worker/ReaderCacheTest.java   |  136 ++
 .../runners/dataflow/worker/ReaderFactoryTest.java |  158 ++
 .../runners/dataflow/worker/ReaderTestUtils.java   |  152 ++
 ...ReifyTimestampAndWindowsParDoFnFactoryTest.java |   94 +
 ...essCoderCloudObjectTranslatorRegistrarTest.java |  105 +
 .../dataflow/worker/SdkHarnessRegistryTest.java    |  123 +
 .../dataflow/worker/ShuffleReaderFactoryTest.java  |  205 ++
 .../dataflow/worker/ShuffleSinkFactoryTest.java    |  173 ++
 .../runners/dataflow/worker/ShuffleSinkTest.java   |  248 ++
 .../runners/dataflow/worker/SimpleParDoFnTest.java |  612 +++++
 .../runners/dataflow/worker/SinkRegistryTest.java  |   77 +
 .../worker/SourceOperationExecutorFactoryTest.java |   80 +
 .../runners/dataflow/worker/StateFetcherTest.java  |  333 +++
 .../worker/StepContextStateInternalsFactory.java   |   41 +
 .../worker/StreamingDataflowWorkerTest.java        | 2568 ++++++++++++++++++++
 .../worker/StreamingGroupAlsoByWindowFnsTest.java  |  662 +++++
 ...reamingGroupAlsoByWindowsReshuffleDoFnTest.java |  220 ++
 ...eamingKeyedWorkItemSideInputDoFnRunnerTest.java |  206 ++
 .../worker/StreamingModeExecutionContextTest.java  |  376 +++
 ...eamingPCollectionViewWriterDoFnFactoryTest.java |   72 +
 .../worker/StreamingSideInputDoFnRunnerTest.java   |  459 ++++
 .../worker/StreamingSideInputFetcherTest.java      |  214 ++
 .../worker/StreamingStepMetricsContainerTest.java  |  179 ++
 .../dataflow/worker/TestOperationContext.java      |  156 ++
 .../worker/TestShuffleReadCounterFactory.java      |   62 +
 .../runners/dataflow/worker/TestShuffleReader.java |  160 ++
 .../dataflow/worker/TestShuffleReaderTest.java     |  169 ++
 .../runners/dataflow/worker/TestShuffleWriter.java |   81 +
 .../ToIsmRecordForMultimapDoFnFactoryTest.java     |  107 +
 .../worker/UngroupedShuffleReaderTest.java         |  133 +
 .../dataflow/worker/UserParDoFnFactoryTest.java    |  451 ++++
 .../dataflow/worker/ValuesDoFnFactoryTest.java     |   57 +
 .../dataflow/worker/WindmillKeyedWorkItemTest.java |  186 ++
 .../worker/WindmillReaderIteratorBaseTest.java     |   95 +
 .../dataflow/worker/WindmillStateCacheTest.java    |  235 ++
 .../worker/WindmillStateInternalsTest.java         | 1019 ++++++++
 .../dataflow/worker/WindmillStateReaderTest.java   |  406 ++++
 .../dataflow/worker/WindmillStateTestUtils.java    |   81 +
 .../dataflow/worker/WindmillTimeUtilsTest.java     |   71 +
 .../worker/WindmillTimerInternalsTest.java         |  109 +
 .../dataflow/worker/WorkItemStatusClientTest.java  |  561 +++++
 .../WorkerCustomSourcesSplitOnlySourceTest.java    |   96 +
 .../dataflow/worker/WorkerCustomSourcesTest.java   |  811 +++++++
 .../worker/WorkerPipelineOptionsFactoryTest.java   |   83 +
 .../worker/WorkerUncaughtExceptionHandlerTest.java |   45 +
 .../runners/dataflow/worker/apiary/ApiaryTest.java |   47 +
 ...FixMultiOutputInfosOnParDoInstructionsTest.java |  105 +
 .../worker/counters/CounterFactoryTest.java        |   76 +
 .../dataflow/worker/counters/CounterSetTest.java   |  162 ++
 .../dataflow/worker/counters/CounterTest.java      |  475 ++++
 .../worker/fn/BeamFnControlServiceTest.java        |  134 +
 .../dataflow/worker/fn/IdGeneratorTest.java        |   39 +
 .../dataflow/worker/fn/ServerFactoryTest.java      |  245 ++
 .../worker/fn/SocketAddressFactoryTest.java        |   56 +
 .../fn/control/BeamFnMapTaskExecutorTest.java      |  440 ++++
 .../RegisterAndProcessBundleOperationTest.java     |  852 +++++++
 .../SingularProcessBundleProgressTrackerTest.java  |  141 ++
 .../worker/fn/data/BeamFnDataGrpcServiceTest.java  |  313 +++
 .../fn/data/RemoteGrpcPortReadOperationTest.java   |  155 ++
 .../fn/data/RemoteGrpcPortWriteOperationTest.java  |  151 ++
 .../fn/logging/BeamFnLoggingServiceTest.java       |  224 ++
 .../fn/stream/ServerStreamObserverFactoryTest.java |   80 +
 .../graph/CloneAmbiguousFlattensFunctionTest.java  |  392 +++
 .../CreateRegisterFnOperationFunctionTest.java     |  556 +++++
 .../graph/DeduceFlattenLocationsFunctionTest.java  |  399 +++
 .../graph/DeduceNodeLocationsFunctionTest.java     |  333 +++
 .../runners/dataflow/worker/graph/EdgesTest.java   |   58 +
 ...tFetchAndFilterStreamingSideInputNodesTest.java |  262 ++
 .../graph/LengthPrefixUnknownCodersTest.java       |  357 +++
 .../worker/graph/MapTaskToNetworkFunctionTest.java |  435 ++++
 .../dataflow/worker/graph/NetworksTest.java        |  256 ++
 .../runners/dataflow/worker/graph/NodesTest.java   |  154 ++
 .../RemoveFlattenInstructionsFunctionTest.java     |  360 +++
 .../ReplacePgbkWithPrecombineFunctionTest.java     |  154 ++
 .../logging/DataflowWorkerLoggingHandlerTest.java  |  362 +++
 .../DataflowWorkerLoggingInitializerTest.java      |  281 +++
 .../JulHandlerPrintStreamAdapterFactoryTest.java   |  122 +
 .../worker/profiler/ScopedProfilerTest.java        |  146 ++
 .../dataflow/worker/status/DebugCaptureTest.java   |  145 ++
 .../dataflow/worker/status/ThreadzServletTest.java |   67 +
 .../worker/status/WorkerStatusPagesTest.java       |   85 +
 .../worker/testing/RestoreDataflowLoggingMDC.java  |   53 +
 .../testing/RestoreDataflowLoggingMDCTest.java     |   85 +
 .../worker/testing/TestCountingSource.java         |  243 ++
 .../worker/util/BatchGroupAlsoByWindowFnsTest.java |  154 ++
 .../BatchGroupAlsoByWindowReshuffleDoFnTest.java   |  204 ++
 .../dataflow/worker/util/CloudSourceUtilsTest.java |   83 +
 ...gGroupAlsoByWindowsViaOutputBufferDoFnTest.java |  105 +
 .../worker/util/CounterHamcrestMatchers.java       |  395 +++
 .../util/DataflowCounterUpdateExtractorTest.java   |  314 +++
 .../dataflow/worker/util/FluentBackoffTest.java    |  218 ++
 .../worker/util/GroupAlsoByWindowProperties.java   |  637 +++++
 .../GroupAlsoByWindowViaIteratorsDoFnTest.java     |  100 +
 .../GroupAlsoByWindowViaOutputBufferDoFnTest.java  |  109 +
 .../util/GroupAlsoByWindowsAndCombineDoFnTest.java |   81 +
 .../dataflow/worker/util/ListOutputManager.java    |   59 +
 .../dataflow/worker/util/MemoryMonitorTest.java    |  149 ++
 .../worker/util/ScalableBloomFilterTest.java       |  133 +
 .../dataflow/worker/util/TimerOrElementTest.java   |   68 +
 .../util/common/TaggedReiteratorListTest.java      |  223 ++
 .../worker/BatchingShuffleEntryReaderTest.java     |  137 ++
 .../worker/CachingShuffleBatchReaderTest.java      |   93 +
 .../common/worker/ExecutionStateSamplerTest.java   |  209 ++
 .../util/common/worker/ExecutorTestUtils.java      |  179 ++
 .../util/common/worker/FlattenOperationTest.java   |  107 +
 .../worker/GroupingShuffleEntryIteratorTest.java   |  289 +++
 .../worker/GroupingShuffleRangeTrackerTest.java    |  176 ++
 .../util/common/worker/GroupingTablesTest.java     |  338 +++
 .../util/common/worker/MapTaskExecutorTest.java    |  484 ++++
 .../worker/OutputObjectAndByteCounterTest.java     |  134 +
 .../util/common/worker/OutputReceiverTest.java     |   72 +
 .../util/common/worker/ParDoOperationTest.java     |  171 ++
 .../util/common/worker/ReadOperationTest.java      |  585 +++++
 .../util/common/worker/ShuffleEntryTest.java       |  147 ++
 .../worker/util/common/worker/StubbedExecutor.java |   80 +
 .../util/common/worker/TestOutputReceiver.java     |  145 ++
 .../common/worker/WorkProgressUpdaterTest.java     |  349 +++
 .../util/common/worker/WriteOperationTest.java     |  183 ++
 .../worker/windmill/GrpcWindmillServerTest.java    |  714 ++++++
 .../worker/windmill/build.gradle                   |   31 +
 .../worker/windmill/src/main/proto/pubsub.proto    |   38 +
 .../worker/windmill/src/main/proto/windmill.proto  |  502 ++++
 .../windmill/src/main/proto/windmill_service.proto |   50 +
 settings.gradle                                    |    4 +
 454 files changed, 92767 insertions(+)

diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle
new file mode 100644
index 0000000..d28b162
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/build.gradle
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+/******************************************************************************/
+// Apply BeamModulePlugin
+
+// Reuse project_root/buildSrc in this build.gradle file to reduce the
+// maintenance burden and simpily this file. See BeamModulePlugin for
+// documentation on default build tasks and properties that are enabled in
+// addition to natures that will be applied to worker.
+apply plugin: org.apache.beam.gradle.BeamModulePlugin
+
+group = "org.apache.beam.runners.dataflow"
+
+/******************************************************************************/
+// Apply Java nature with customized configurations
+
+// Set a specific version of 'com.google.apis:google-api-services-dataflow'
+// by adding -Pdataflow.version=<version> in Gradle command. Otherwise,
+// 'google_clients_version' defined in BeamModulePlugin will be used as default.
+def DATAFLOW_VERSION = "dataflow.version"
+
+// To build FnAPI or legacy worker.
+// Use -PisLegacyWorker in Gradle command if build legacy worker, otherwise,
+// FnAPI worker is considered as default.
+def is_legacy_worker = {
+  return project.hasProperty("isLegacyWorker")
+}
+
+// Get full dependency of 'com.google.apis:google-api-services-dataflow'
+def google_api_services_dataflow = project.hasProperty(DATAFLOW_VERSION) ? "com.google.apis:google-api-services-dataflow:" + getProperty(DATAFLOW_VERSION) : library.java.google_api_services_dataflow
+
+// Returns a string representing the relocated path to be used with the shadow
+// plugin when given a suffix such as "com.".
+def getWorkerRelocatedPath = { String suffix ->
+  return ("org.apache.beam.runners.dataflow.worker.repackaged."
+          + suffix)
+}
+
+// Following listed dependencies will be shaded only in fnapi worker, not legacy
+// worker
+def sdk_provided_dependencies = [
+  "org.apache.beam:beam-runners-google-cloud-dataflow-java:$version",
+  "org.apache.beam:beam-sdks-java-core:$version",
+  "org.apache.beam:beam-sdks-java-extensions-google-cloud-platform-core:$version",
+  "org.apache.beam:beam-sdks-java-io-google-cloud-platform:$version",
+  google_api_services_dataflow,
+  library.java.avro,
+  library.java.google_api_client,
+  library.java.google_http_client,
+  library.java.google_http_client_jackson,
+  library.java.jackson_annotations,
+  library.java.jackson_core,
+  library.java.jackson_databind,
+  library.java.joda_time,
+]
+
+// Exclude unneeded dependencies when building jar
+def excluded_dependencies = [
+  "com.google.auto.service:auto-service",      // Provided scope added from applyJavaNature
+  "com.google.auto.value:auto-value",          // Provided scope added from applyJavaNature
+  "org.codehaus.jackson:jackson-core-asl",     // Exclude an old version of jackson-core-asl introduced by google-http-client-jackson
+  "org.objenesis:objenesis",                   // Transitive dependency introduced from Beam
+  "org.tukaani:xz",                            // Transitive dependency introduced from Beam
+  library.java.commons_compress,               // Transitive dependency introduced from Beam
+  library.java.error_prone_annotations,        // Provided scope added in worker
+  library.java.hamcrest_core,                  // Test only
+  library.java.hamcrest_library,               // Test only
+  library.java.junit,                          // Test only
+]
+
+applyJavaNature(validateShadowJar: false, shadowClosure: DEFAULT_SHADOW_CLOSURE << {
+  dependencies {
+    include(project(path: ":beam-runners-google-cloud-dataflow-java-windmill", configuration: "shadow"))
+    include(dependency(".*:.*"))
+
+    if (is_legacy_worker()) {
+      sdk_provided_dependencies.each {
+        exclude(dependency(it))
+      }
+      excluded_dependencies.each {
+        exclude(dependency(it))
+      }
+    }
+  }
+
+  // Archive name pattern: ${name}-${appendix}-${version}-${classifier}.jar
+  appendix = is_legacy_worker() ? "legacy-bundled" : "fnapi-bundled"
+
+  // Include original source files extracted under
+  // '$buildDir/original_sources_to_package' to jar
+  from "$buildDir/original_sources_to_package"
+
+  exclude "META-INF/LICENSE.txt"
+  exclude "about.html"
+  exclude "google/protobuf/*.proto"
+  exclude "windmill*.proto"
+
+  if (is_legacy_worker()) {
+    relocate("com.", getWorkerRelocatedPath("com.")) {
+      exclude "com.fasterxml.jackson.**"
+      exclude "com.google.api.client.**"
+      exclude "com.google.api.services.bigquery.**"
+      exclude "com.google.api.services.clouddebugger.**"
+      exclude "com.google.api.services.dataflow.**"
+      exclude "com.google.api.services.datastore.**"
+      exclude "com.google.api.services.pubsub.**"
+      exclude "com.google.api.services.storage.**"
+      exclude "com.google.auth.**"
+      exclude "com.google.cloud.dataflow.**"
+      exclude "com.sun.management*"
+      exclude "com.sun.management.**"
+    }
+    relocate("javax.servlet", getWorkerRelocatedPath("javax.servlet"))
+    relocate("io.", getWorkerRelocatedPath("io."))
+    relocate("okio.", getWorkerRelocatedPath("okio."))
+    relocate("org.", getWorkerRelocatedPath("org.")) {
+      // Exclude netty-tcnative from shading since gRPC relies on Netty to be able
+      // to load org.apache.tomcat.jni.SSL to provide an SSL context.
+      exclude "org.apache.avro.**"
+      exclude "org.apache.beam.**"
+      exclude "org.apache.tomcat.jni.**"
+      exclude "org.conscrypt.**"
+      exclude "org.eclipse.jetty.alpn.**"
+      exclude "org.eclipse.jetty.npn.**"
+      exclude "org.hamcrest.**"
+      exclude "org.joda.time.**"
+      exclude "org.junit.**"
+      exclude "org.slf4j.**"
+      exclude "org.w3c.dom.**"
+    }
+    relocate("org.apache.beam.runners.core.construction.",
+             getWorkerRelocatedPath("org."))
+  }
+})
+
+/******************************************************************************/
+// Configure the worker root project
+
+configurations {
+  sourceFile
+
+  // Ban these dependencies from all configurations
+  all {
+    // Ban the usage of AppleJavaExtensions in findbugs.
+    exclude group: "com.apple", module: "AppleJavaExtensions"
+  }
+}
+
+dependencies {
+  // Set dependencies to shadow scope by default, but in a property so they can
+  // be downgraded when building a legacy (non-FnAPI) worker.
+  if (is_legacy_worker()) {
+    sdk_provided_dependencies.each {
+      provided(it)
+    }
+  } else {
+    sdk_provided_dependencies.each {
+      shadow(it)
+    }
+  }
+
+  compile "org.apache.beam:beam-model-fn-execution:$version"
+  compile "org.apache.beam:beam-model-pipeline:$version"
+  compile "org.apache.beam:beam-runners-core-construction-java:$version"
+  compile "org.apache.beam:beam-runners-core-java:$version"
+  compile "org.apache.beam:beam-runners-java-fn-execution:$version"
+  compile "org.apache.beam:beam-sdks-java-fn-execution:$version"
+  compile project(path: ":beam-runners-google-cloud-dataflow-java-windmill", configuration: "shadow")
+
+  compile library.java.guava
+  compile library.java.slf4j_api
+  compile "javax.servlet:javax.servlet-api:3.1.0"
+  compile "org.conscrypt:conscrypt-openjdk:1.1.3:linux-x86_64"
+  compile "org.eclipse.jetty:jetty-server:9.2.10.v20150310"
+  compile "org.eclipse.jetty:jetty-servlet:9.2.10.v20150310"
+
+  provided library.java.error_prone_annotations
+
+  runtime library.java.slf4j_jdk14
+
+  testCompile "org.apache.beam:beam-runners-core-java:$version:tests"
+  testCompile "org.apache.beam:beam-runners-direct-java:$version"
+  testCompile "org.apache.beam:beam-sdks-java-core:$version:tests"
+  testCompile "org.apache.beam:beam-sdks-java-extensions-google-cloud-platform-core:$version:tests"
+
+  testCompile library.java.guava_testlib
+  testCompile library.java.hamcrest_core
+  testCompile library.java.hamcrest_library
+  testCompile library.java.junit
+  testCompile library.java.mockito_core
+}
+
+//TODO(BEAM-5657): checktyle task should be enabled in the future.
+checkstyleMain.enabled = false
+checkstyleTest.enabled = false
+//TODO(BEAM-5658): fingbugs task should be enabled in the future.
+findbugsMain.enabled = false
+//TODO(BEAM-5659): javadoc task should be enabled in the future.
+javadoc.enabled = false
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingBiConsumer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingBiConsumer.java
new file mode 100644
index 0000000..cae604f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingBiConsumer.java
@@ -0,0 +1,32 @@
+/*
+ * 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.dataflow.harness.util;
+
+import java.util.function.BiConsumer;
+
+/**
+ * A {@link BiConsumer} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8 functional
+ * interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingBiConsumer<T1, T2> {
+  void accept(T1 t1, T2 t2) throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingBiFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingBiFunction.java
new file mode 100644
index 0000000..0d1be33
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingBiFunction.java
@@ -0,0 +1,32 @@
+/*
+ * 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.dataflow.harness.util;
+
+import java.util.function.BiFunction;
+
+/**
+ * A {@link BiFunction} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8 functional
+ * interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingBiFunction<T1, T2, T3> {
+  T3 apply(T1 t1, T2 t2) throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingConsumer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingConsumer.java
new file mode 100644
index 0000000..856de14
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingConsumer.java
@@ -0,0 +1,32 @@
+/*
+ * 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.dataflow.harness.util;
+
+import java.util.function.Consumer;
+
+/**
+ * A {@link Consumer} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8 functional
+ * interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingConsumer<T> {
+  void accept(T t) throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingFunction.java
new file mode 100644
index 0000000..5692918
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingFunction.java
@@ -0,0 +1,32 @@
+/*
+ * 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.dataflow.harness.util;
+
+import java.util.function.Function;
+
+/**
+ * A {@link Function} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8 functional
+ * interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingFunction<T1, T2> {
+  T2 apply(T1 value) throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingRunnable.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingRunnable.java
new file mode 100644
index 0000000..eab3098
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/ThrowingRunnable.java
@@ -0,0 +1,30 @@
+/*
+ * 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.dataflow.harness.util;
+
+/**
+ * A {@link Runnable} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8 functional
+ * interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingRunnable {
+  void run() throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/package-info.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/package-info.java
new file mode 100644
index 0000000..541fcd3
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/harness/util/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * Classes forked from the Java SDK Harness as a temporary workaround before modules are properly
+ * configured. This package should be deleted once the SDK harness utilities that are also useful
+ * for Java runner harnesses are in a common, shared library.
+ */
+package org.apache.beam.runners.dataflow.harness.util;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCounters.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCounters.java
new file mode 100644
index 0000000..527fe22
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCounters.java
@@ -0,0 +1,79 @@
+/*
+ * 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.dataflow.worker;
+
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.CounterName;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+
+/**
+ * ApplianceShuffleCounters implements functionality needed for importing counters from native
+ * Appliance Shuffle code into Java.
+ */
+public final class ApplianceShuffleCounters {
+  /** Factory fro creating counters. */
+  private final CounterFactory counterFactory;
+
+  /** Prefix string used for the counters. */
+  private final String counterPrefix;
+
+  /**
+   * @param counterFactory factory that can be used to create/reuse counters
+   * @param nameContext details about the operation these counters are for
+   */
+  public ApplianceShuffleCounters(
+      CounterFactory counterFactory, NameContext nameContext, String datasetId) {
+    this.counterFactory = counterFactory;
+    this.counterPrefix =
+        String.format("%s-%s-%s-", nameContext.stageName(), nameContext.systemName(), datasetId);
+  }
+
+  /**
+   * Invoked to import a set of counter deltas. All the arrays passed in as parameters are required
+   * to have the same length.
+   *
+   * @param counterNames names of counters to import
+   * @param counterKinds kinds of counters to import ("sum", "max", or "min")
+   * @param counterDeltas counter deltas to import
+   */
+  public void importCounters(String[] counterNames, String[] counterKinds, long[] counterDeltas) {
+    final int length = counterNames.length;
+    if (counterKinds.length != length || counterDeltas.length != length) {
+      throw new AssertionError("array lengths do not match");
+    }
+    for (int i = 0; i < length; ++i) {
+      final CounterName name = CounterName.named(counterPrefix + counterNames[i]);
+      final String kind = counterKinds[i];
+      final long delta = counterDeltas[i];
+      switch (kind) {
+        case "sum":
+          counterFactory.longSum(name).addValue(delta);
+          break;
+        case "max":
+          counterFactory.longMax(name).addValue(delta);
+          break;
+        case "min":
+          counterFactory.longMin(name).addValue(delta);
+          break;
+        default:
+          throw new IllegalArgumentException("unsupported counter kind: " + kind);
+      }
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java
new file mode 100644
index 0000000..64c17b9
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java
@@ -0,0 +1,69 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.base.Preconditions;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.BatchingShuffleEntryReader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.CachingShuffleBatchReader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleBatchReader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleEntry;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleEntryReader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShufflePosition;
+import org.apache.beam.sdk.util.common.Reiterator;
+
+/** An implementation of ShuffleEntryReader that uses ApplianceShuffleReader. */
+public class ApplianceShuffleEntryReader implements ShuffleEntryReader {
+  private ApplianceShuffleReader applianceShuffleReader;
+  private ShuffleEntryReader entryReader;
+
+  public ApplianceShuffleEntryReader(
+      byte[] shuffleReaderConfig,
+      BatchModeExecutionContext executionContext,
+      DataflowOperationContext operationContext,
+      boolean cache) {
+    Preconditions.checkArgument(shuffleReaderConfig != null);
+    applianceShuffleReader = new ApplianceShuffleReader(shuffleReaderConfig, operationContext);
+
+    ShuffleBatchReader batchReader =
+        new ChunkingShuffleBatchReader(executionContext, operationContext, applianceShuffleReader);
+
+    if (cache) {
+      // Limit the size of the cache.
+      final int maxBatches = 32;
+      batchReader = new CachingShuffleBatchReader(batchReader, maxBatches);
+    }
+    entryReader = new BatchingShuffleEntryReader(batchReader);
+  }
+
+  @Override
+  public Reiterator<ShuffleEntry> read(
+      @Nullable ShufflePosition startPosition, @Nullable ShufflePosition endPosition) {
+    return entryReader.read(startPosition, endPosition);
+  }
+
+  @Override
+  public void close() {
+    applianceShuffleReader.close();
+  }
+
+  public String getDatasetId() {
+    return applianceShuffleReader.getDatasetId();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java
new file mode 100644
index 0000000..f42766b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java
@@ -0,0 +1,98 @@
+/*
+ * 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.dataflow.worker;
+
+import java.io.Closeable;
+import java.io.IOException;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OperationContext;
+
+/**
+ * ApplianceShuffleReader reads chunks of data from a shuffle dataset for a position range.
+ *
+ * <p>It is a JNI wrapper of an equivalent C++ class.
+ */
+@ThreadSafe
+public final class ApplianceShuffleReader implements ShuffleReader, Closeable {
+  static {
+    ShuffleLibrary.load();
+  }
+
+  /** Pointer to the underlying C++ ShuffleReader object. */
+  private long nativePointer;
+
+  /** Adapter used to import counterFactory from native code. */
+  private ApplianceShuffleCounters counters;
+
+  /** Whether the underlying C++ object was already destroyed. */
+  private boolean destroyed = false;
+
+  /** Returns the total number of allocated native ShuffleReader objects. */
+  public static native int getNumReaders();
+
+  /**
+   * @param shuffleReaderConfig opaque configuration for creating a shuffle reader
+   * @param operationContext context of the read operation this shuffle reader is associated with
+   */
+  public ApplianceShuffleReader(byte[] shuffleReaderConfig, OperationContext operationContext) {
+    this.nativePointer = createFromConfig(shuffleReaderConfig);
+    this.counters =
+        new ApplianceShuffleCounters(
+            operationContext.counterFactory(), operationContext.nameContext(), getDatasetId());
+  }
+
+  public native String getDatasetId();
+
+  @Override
+  public native ReadChunkResult readIncludingPosition(byte[] startPosition, byte[] endPosition)
+      throws IOException;
+
+  /**
+   * Releases resources associated with this reader.
+   *
+   * <p>May be called multiple times, but once it's called, no other methods can be called on this
+   * object.
+   */
+  @Override
+  public void close() {
+    destroySynchronized();
+  }
+
+  @Override
+  public void finalize() {
+    destroySynchronized();
+  }
+
+  /**
+   * Native methods for interacting with the underlying native shuffle client code. {@code
+   * createFromConfig()} returns a pointer to a newly created C++ ShuffleReader object.
+   */
+  private native long createFromConfig(byte[] shuffleReaderConfig);
+
+  private native void destroy();
+
+  /** Destroys the underlying C++ object if it hasn't yet been destroyed. */
+  private synchronized void destroySynchronized() {
+    if (!destroyed) {
+      destroy();
+      destroyed = true;
+      nativePointer = 0;
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java
new file mode 100644
index 0000000..9fb1def
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.dataflow.worker;
+
+import java.io.IOException;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OperationContext;
+
+/**
+ * ApplianceShuffleWriter writes chunks of data to a shuffle dataset.
+ *
+ * <p>It is a JNI wrapper of an equivalent C++ class.
+ */
+@ThreadSafe
+public final class ApplianceShuffleWriter implements ShuffleWriter {
+  static {
+    ShuffleLibrary.load();
+  }
+
+  /** Pointer to the underlying native shuffle writer code. */
+  private long nativePointer;
+
+  /** Adapter used to import counters from native code. */
+  private ApplianceShuffleCounters counters;
+
+  /**
+   * @param shuffleWriterConfig opaque configuration for creating a shuffle writer
+   * @param bufferSize the writer buffer size
+   * @param operationContext context of the write operation this shuffle writer is associated with
+   */
+  public ApplianceShuffleWriter(
+      byte[] shuffleWriterConfig, long bufferSize, OperationContext operationContext) {
+    this.nativePointer = createFromConfig(shuffleWriterConfig, bufferSize);
+    this.counters =
+        new ApplianceShuffleCounters(
+            operationContext.counterFactory(), operationContext.nameContext(), getDatasetId());
+  }
+
+  @Override
+  public void finalize() {
+    destroy();
+  }
+
+  /** Native methods for interacting with the underlying native shuffle writer code. */
+  private native long createFromConfig(byte[] shuffleWriterConfig, long bufferSize);
+
+  private native void destroy();
+
+  public native String getDatasetId();
+
+  @Override
+  public native void write(byte[] chunk) throws IOException;
+
+  @Override
+  public native void close() throws IOException;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java
new file mode 100644
index 0000000..3e4f100
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java
@@ -0,0 +1,138 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.beam.runners.dataflow.util.Structs.getBytes;
+
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.collect.Iterables;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.PropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Instant;
+
+/**
+ * A {@link ParDoFnFactory} to create instances of AssignWindowsDoFn according to specifications
+ * from the Dataflow service.
+ */
+class AssignWindowsParDoFnFactory implements ParDoFnFactory {
+  @Override
+  public ParDoFn create(
+      PipelineOptions options,
+      CloudObject cloudUserFn,
+      List<SideInputInfo> sideInputInfos,
+      TupleTag<?> mainOutputTag,
+      Map<TupleTag<?>, Integer> outputTupleTagsToReceiverIndices,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+
+    byte[] encodedWindowingStrategy = getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN);
+
+    WindowingStrategy<?, ?> deserializedWindowingStrategy =
+        GroupAlsoByWindowParDoFnFactory.deserializeWindowingStrategy(encodedWindowingStrategy);
+
+    @SuppressWarnings("unchecked")
+    WindowingStrategy<Object, BoundedWindow> windowingStrategy =
+        (WindowingStrategy<Object, BoundedWindow>) deserializedWindowingStrategy;
+
+    return new AssignWindowsParDoFn<>(
+        windowingStrategy.getWindowFn(), executionContext.getStepContext(operationContext));
+  }
+
+  private static class AssignWindowsParDoFn<T, W extends BoundedWindow> implements ParDoFn {
+    private final DataflowExecutionContext.DataflowStepContext stepContext;
+    private final WindowFn<T, W> windowFn;
+
+    @Nullable private Receiver receiver;
+
+    AssignWindowsParDoFn(
+        WindowFn<T, W> windowFn, DataflowExecutionContext.DataflowStepContext stepContext) {
+      this.stepContext = stepContext;
+      this.windowFn = windowFn;
+    }
+
+    @Override
+    public void startBundle(Receiver... receivers) throws Exception {
+      checkState(
+          receivers.length == 1,
+          "%s.startBundle() called with %s receivers, expected exactly 1. "
+              + "This is a bug in the Dataflow service",
+          getClass().getSimpleName(),
+          receivers.length);
+      receiver = receivers[0];
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void processElement(Object untypedElem) throws Exception {
+      WindowedValue<T> elem = (WindowedValue<T>) untypedElem;
+
+      Collection<W> windows =
+          windowFn.assignWindows(
+              windowFn.new AssignContext() {
+                @Override
+                public T element() {
+                  return elem.getValue();
+                }
+
+                @Override
+                public Instant timestamp() {
+                  return elem.getTimestamp();
+                }
+
+                @Override
+                public BoundedWindow window() {
+                  return Iterables.getOnlyElement(elem.getWindows());
+                }
+              });
+
+      WindowedValue<T> res =
+          WindowedValue.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPane());
+      receiver.process(res);
+    }
+
+    @Override
+    public void processTimers() throws Exception {
+      // Nothing.
+    }
+
+    @Override
+    public void finishBundle() throws Exception {
+      receiver = null;
+    }
+
+    @Override
+    public void abort() throws Exception {
+      receiver = null;
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java
new file mode 100644
index 0000000..c710722
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java
@@ -0,0 +1,179 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.ApproximateReportedProgress;
+import com.google.api.services.dataflow.model.ApproximateSplitRequest;
+import com.google.api.services.dataflow.model.ReportedParallelism;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import org.apache.avro.Schema;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.AvroSource;
+import org.apache.beam.sdk.io.AvroSource.AvroReader;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.OffsetBasedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A source that reads PCollections that have been materialized as Avro files. Records are read from
+ * the Avro file as a series of byte arrays. The coder provided is used to deserialize each record
+ * from a byte array.
+ *
+ * @param <T> the type of the elements read from the source
+ */
+public class AvroByteReader<T> extends NativeReader<T> {
+  private static final Logger LOG = LoggerFactory.getLogger(AvroByteReader.class);
+
+  final long startPosition;
+  final long endPosition;
+  final String filename;
+  final AvroSource<ByteBuffer> avroSource;
+  final PipelineOptions options;
+
+  final Coder<T> coder;
+  private final Schema schema = Schema.create(Schema.Type.BYTES);
+
+  public AvroByteReader(
+      String filename,
+      long startPosition,
+      long endPosition,
+      Coder<T> coder,
+      PipelineOptions options) {
+    this.filename = checkNotNull(filename, "filename");
+    this.startPosition = startPosition;
+    this.endPosition = endPosition;
+    this.coder = checkNotNull(coder, "coder");
+    this.options = checkNotNull(options, "options");
+    this.avroSource =
+        (AvroSource<ByteBuffer>) ((AvroSource) AvroSource.from(filename).withSchema(schema));
+  }
+
+  @Override
+  public AvroByteFileIterator iterator() throws IOException {
+    BoundedSource.BoundedReader<ByteBuffer> reader;
+    if (startPosition == 0 && endPosition == Long.MAX_VALUE) {
+      // Read entire file (or collection of files).
+      reader = avroSource.createReader(options);
+    } else {
+      // Read a subrange of file.
+      reader =
+          avroSource
+              .createForSubrangeOfFile(
+                  FileSystems.matchSingleFileSpec(filename), startPosition, endPosition)
+              .createReader(options);
+    }
+    return new AvroByteFileIterator((AvroReader<ByteBuffer>) reader);
+  }
+
+  class AvroByteFileIterator extends NativeReaderIterator<T> {
+    private final AvroReader<ByteBuffer> reader;
+    private Optional<T> current;
+
+    public AvroByteFileIterator(AvroReader<ByteBuffer> reader) {
+      this.reader = reader;
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      if (!reader.start()) {
+        current = Optional.empty();
+        return false;
+      }
+      updateCurrent();
+      return true;
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      if (!reader.advance()) {
+        current = Optional.empty();
+        return false;
+      }
+      updateCurrent();
+      return true;
+    }
+
+    private void updateCurrent() throws IOException {
+      ByteBuffer inBuffer = reader.getCurrent();
+      notifyElementRead(inBuffer.remaining());
+      byte[] encodedElem = new byte[inBuffer.remaining()];
+      inBuffer.get(encodedElem);
+      current = Optional.of(CoderUtils.decodeFromByteArray(coder, encodedElem));
+    }
+
+    @Override
+    public T getCurrent() throws NoSuchElementException {
+      return current.get();
+    }
+
+    @Override
+    public Progress getProgress() {
+      Double readerProgress = reader.getFractionConsumed();
+      if (readerProgress == null) {
+        return null;
+      }
+      ApproximateReportedProgress progress = new ApproximateReportedProgress();
+      progress.setFractionConsumed(readerProgress);
+      double consumedParallelism = reader.getSplitPointsConsumed();
+      double remainingParallelism = reader.getSplitPointsRemaining();
+      progress.setConsumedParallelism(new ReportedParallelism().setValue(consumedParallelism));
+      if (remainingParallelism >= 0) {
+        progress.setRemainingParallelism(new ReportedParallelism().setValue(remainingParallelism));
+      }
+      return SourceTranslationUtils.cloudProgressToReaderProgress(progress);
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) {
+      ApproximateSplitRequest splitProgress =
+          SourceTranslationUtils.splitRequestToApproximateSplitRequest(splitRequest);
+      double splitAtFraction = splitProgress.getFractionConsumed();
+      LOG.info("Received request for dynamic split at {}", splitAtFraction);
+      OffsetBasedSource<ByteBuffer> residual = reader.splitAtFraction(splitAtFraction);
+      if (residual == null) {
+        LOG.info("Rejected split request for split at {}", splitAtFraction);
+        return null;
+      }
+      com.google.api.services.dataflow.model.Position acceptedPosition =
+          new com.google.api.services.dataflow.model.Position();
+      acceptedPosition.setByteOffset(residual.getStartOffset());
+      LOG.info(
+          "Accepted split for position {} which resulted in a new source with byte offset {}",
+          splitAtFraction,
+          residual.getStartOffset());
+      return new DynamicSplitResultWithPosition(
+          SourceTranslationUtils.cloudPositionToReaderPosition(acceptedPosition));
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java
new file mode 100644
index 0000000..d4b3fdc
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.dataflow.worker;
+
+import static org.apache.beam.runners.dataflow.util.Structs.getLong;
+import static org.apache.beam.runners.dataflow.util.Structs.getString;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Creates an {@link AvroByteReader} from a CloudObject spec. */
+public class AvroByteReaderFactory implements ReaderFactory {
+
+  /** A {@link ReaderFactory.Registrar} for Avro byte sources. */
+  @AutoService(ReaderFactory.Registrar.class)
+  public static class Registrar implements ReaderFactory.Registrar {
+
+    @Override
+    public Map<String, ReaderFactory> factories() {
+      return ImmutableMap.of("AvroSource", new AvroByteReaderFactory());
+    }
+  }
+
+  public AvroByteReaderFactory() {}
+
+  @Override
+  public NativeReader<?> create(
+      CloudObject spec,
+      @Nullable Coder<?> coder,
+      @Nullable PipelineOptions options,
+      @Nullable DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    return create(spec, coder, options);
+  }
+
+  NativeReader<?> create(CloudObject spec, Coder<?> coder, PipelineOptions options)
+      throws Exception {
+    String filename = getString(spec, WorkerPropertyNames.FILENAME);
+    long startOffset = getLong(spec, WorkerPropertyNames.START_OFFSET, 0L);
+    long endOffset = getLong(spec, WorkerPropertyNames.END_OFFSET, Long.MAX_VALUE);
+    return new AvroByteReader<>(filename, startOffset, endOffset, coder, options);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteSink.java
new file mode 100644
index 0000000..96a603a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteSink.java
@@ -0,0 +1,87 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Sink;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.MimeTypes;
+
+/**
+ * A sink that writes Avro files. Records are written to the Avro file as a series of byte arrays.
+ * The coder provided is used to serialize each record into a byte array.
+ *
+ * @param <T> the type of the elements written to the sink
+ */
+public class AvroByteSink<T> extends Sink<T> {
+
+  final Coder<T> coder;
+  @VisibleForTesting final ResourceId resourceId;
+  private final Schema schema = Schema.create(Schema.Type.BYTES);
+
+  public AvroByteSink(ResourceId resourceId, Coder<T> coder) {
+    this.coder = coder;
+    this.resourceId = resourceId;
+  }
+
+  @Override
+  public SinkWriter<T> writer() throws IOException {
+    return new AvroByteFileWriter();
+  }
+
+  class AvroByteFileWriter implements SinkWriter<T> {
+    DataFileWriter<ByteBuffer> fileWriter;
+
+    public AvroByteFileWriter() throws IOException {
+      WritableByteChannel writer = FileSystems.create(resourceId, MimeTypes.BINARY);
+
+      fileWriter = new DataFileWriter<>(new GenericDatumWriter<ByteBuffer>(schema));
+      fileWriter.create(schema, Channels.newOutputStream(writer));
+    }
+
+    @Override
+    public long add(T value) throws IOException {
+      byte[] encodedElem = CoderUtils.encodeToByteArray(coder, value);
+      ByteBuffer encodedBuffer = ByteBuffer.wrap(encodedElem);
+      fileWriter.append(encodedBuffer);
+      return encodedElem.length;
+    }
+
+    @Override
+    public void close() throws IOException {
+      fileWriter.close();
+    }
+
+    @Override
+    public void abort() throws IOException {
+      // DataFileWriter implements Closeable and is idempotent.
+      fileWriter.close();
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactory.java
new file mode 100644
index 0000000..bdc740d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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.dataflow.worker;
+
+import static org.apache.beam.runners.dataflow.util.Structs.getString;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Sink;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Creates an {@link AvroByteSink} from a {@link CloudObject} spec. */
+public final class AvroByteSinkFactory implements SinkFactory {
+
+  /** A {@link SinkFactory.Registrar} for Avro byte sinks. */
+  @AutoService(SinkFactory.Registrar.class)
+  public static class Registrar implements SinkFactory.Registrar {
+
+    @Override
+    public Map<String, SinkFactory> factories() {
+      return ImmutableMap.of("AvroSink", new AvroByteSinkFactory());
+    }
+  }
+
+  @Override
+  public Sink<?> create(
+      CloudObject spec,
+      Coder<?> coder,
+      @Nullable PipelineOptions options,
+      @Nullable DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    String filename = getString(spec, WorkerPropertyNames.FILENAME);
+    return new AvroByteSink<>(FileSystems.matchNewResource(filename, false), coder);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java
new file mode 100644
index 0000000..40ba53b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java
@@ -0,0 +1,430 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.api.services.dataflow.model.MapTask;
+import com.google.api.services.dataflow.model.WorkItem;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.graph.MutableNetwork;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Function;
+import javax.annotation.Nullable;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.SdkHarnessRegistry.SdkWorkerHarness;
+import org.apache.beam.runners.dataflow.worker.apiary.FixMultiOutputInfosOnParDoInstructions;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import org.apache.beam.runners.dataflow.worker.fn.IdGenerator;
+import org.apache.beam.runners.dataflow.worker.graph.CloneAmbiguousFlattensFunction;
+import org.apache.beam.runners.dataflow.worker.graph.CreateRegisterFnOperationFunction;
+import org.apache.beam.runners.dataflow.worker.graph.DeduceFlattenLocationsFunction;
+import org.apache.beam.runners.dataflow.worker.graph.DeduceNodeLocationsFunction;
+import org.apache.beam.runners.dataflow.worker.graph.Edges.Edge;
+import org.apache.beam.runners.dataflow.worker.graph.LengthPrefixUnknownCoders;
+import org.apache.beam.runners.dataflow.worker.graph.MapTaskToNetworkFunction;
+import org.apache.beam.runners.dataflow.worker.graph.Networks;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.Node;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.RemoteGrpcPortNode;
+import org.apache.beam.runners.dataflow.worker.graph.RegisterNodeFunction;
+import org.apache.beam.runners.dataflow.worker.graph.ReplacePgbkWithPrecombineFunction;
+import org.apache.beam.runners.dataflow.worker.status.DebugCapture;
+import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateSampler;
+import org.apache.beam.sdk.util.Weighted;
+import org.apache.beam.sdk.util.WeightedValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is a semi-abstract harness for executing WorkItem tasks in Java workers. Concrete
+ * implementations need to implement a WorkUnitClient.
+ *
+ * <p>DataflowWorker presents one public interface, getAndPerformWork(), which uses the
+ * WorkUnitClient to get work, execute it, and update the work.
+ */
+public class BatchDataflowWorker implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(BatchDataflowWorker.class);
+
+  /** A client to get and update work items. */
+  private final WorkUnitClient workUnitClient;
+
+  /**
+   * Pipeline options, initially provided via the constructor and partially provided via each work
+   * work unit.
+   */
+  private final DataflowWorkerHarnessOptions options;
+
+  /** The factory to create {@link DataflowMapTaskExecutor DataflowMapTaskExecutors}. */
+  private final DataflowMapTaskExecutorFactory mapTaskExecutorFactory;
+
+  /**
+   * Function which converts map tasks to their network representation for execution.
+   *
+   * <p>It will:
+   *
+   * <ul>
+   *   <li>Fix multi output infos to exist on all ParDo instructions.
+   *   <li>Translate the map task to a network representation.
+   * </ul>
+   */
+  private static final Function<MapTask, MutableNetwork<Node, Edge>> mapTaskToBaseNetwork =
+      new FixMultiOutputInfosOnParDoInstructions(IdGenerator::generate)
+          .andThen(new MapTaskToNetworkFunction());
+
+  /** Registry of known {@link ReaderFactory ReaderFactories}. */
+  private final ReaderRegistry readerRegistry = ReaderRegistry.defaultRegistry();
+
+  /** Registry of known {@link SinkFactory SinkFactories}. */
+  private final SinkRegistry sinkRegistry = SinkRegistry.defaultRegistry();
+
+  /** A side input cache shared between all execution contexts. */
+  private final Cache<?, WeightedValue<?>> sideInputDataCache;
+
+  /**
+   * A side input cache shared between all execution contexts. This cache is meant to store values
+   * as weak references. This allows for insertion of logical keys with zero weight since they will
+   * only be scoped to the lifetime of the value being cached.
+   */
+  private final Cache<?, ?> sideInputWeakReferenceCache;
+
+  private static final int DEFAULT_STATUS_PORT = 18081;
+
+  /** Status pages returning health of worker. */
+  private WorkerStatusPages statusPages;
+
+  /** Periodic sender of debug information to the debug capture service. */
+  private DebugCapture.Manager debugCaptureManager = null;
+
+  /**
+   * A weight in "bytes" for the overhead of a {@link Weighted} wrapper in the cache. It is just an
+   * approximation so it is OK for it to be fairly arbitrary as long as it is nonzero.
+   */
+  private static final int OVERHEAD_WEIGHT = 8;
+
+  private static final long MEGABYTES = 1024 * 1024;
+
+  /**
+   * Limit the number of logical references. Weak references may never be cleared if the object is
+   * long lived irrespective if the user actually is interested in the key lookup anymore.
+   */
+  private static final int MAX_LOGICAL_REFERENCES = 1_000_000;
+
+  /** How many concurrent write operations to a cache should we allow. */
+  private static final int CACHE_CONCURRENCY_LEVEL = 4 * Runtime.getRuntime().availableProcessors();
+
+  private final SdkHarnessRegistry sdkHarnessRegistry;
+  private final Function<MapTask, MutableNetwork<Node, Edge>> mapTaskToNetwork;
+
+  private final MemoryMonitor memoryMonitor;
+  private final Thread memoryMonitorThread;
+
+  /**
+   * Returns a {@link BatchDataflowWorker} configured to execute user functions via intrinsic Java
+   * execution.
+   *
+   * <p>This is also known as the "legacy" or "pre-portability" approach. It is not yet deprecated
+   * as there is not a compatible path forward for users.
+   */
+  static BatchDataflowWorker forBatchIntrinsicWorkerHarness(
+      WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options) {
+    return new BatchDataflowWorker(
+        null,
+        SdkHarnessRegistries.emptySdkHarnessRegistry(),
+        workUnitClient,
+        IntrinsicMapTaskExecutorFactory.defaultFactory(),
+        options);
+  }
+
+  /**
+   * Returns a {@link BatchDataflowWorker} configured to execute user functions via the Beam "Fn
+   * API".
+   *
+   * <p>This is also known as the "portable" or "Beam model" approach.
+   */
+  static BatchDataflowWorker forBatchFnWorkerHarness(
+      @Nullable RunnerApi.Pipeline pipeline,
+      SdkHarnessRegistry sdkHarnessRegistry,
+      WorkUnitClient workUnitClient,
+      DataflowWorkerHarnessOptions options) {
+    return new BatchDataflowWorker(
+        pipeline,
+        sdkHarnessRegistry,
+        workUnitClient,
+        BeamFnMapTaskExecutorFactory.defaultFactory(),
+        options);
+  }
+
+  protected BatchDataflowWorker(
+      @Nullable RunnerApi.Pipeline pipeline,
+      SdkHarnessRegistry sdkHarnessRegistry,
+      WorkUnitClient workUnitClient,
+      DataflowMapTaskExecutorFactory mapTaskExecutorFactory,
+      DataflowWorkerHarnessOptions options) {
+    this.mapTaskExecutorFactory = mapTaskExecutorFactory;
+    this.sdkHarnessRegistry = sdkHarnessRegistry;
+    this.workUnitClient = workUnitClient;
+    this.options = options;
+
+    this.sideInputDataCache =
+        CacheBuilder.newBuilder()
+            .maximumWeight(options.getWorkerCacheMb() * MEGABYTES) // weights are in bytes
+            .weigher(Weighers.fixedWeightKeys(OVERHEAD_WEIGHT))
+            .softValues()
+            .concurrencyLevel(CACHE_CONCURRENCY_LEVEL)
+            .build();
+
+    this.sideInputWeakReferenceCache =
+        CacheBuilder.newBuilder()
+            .maximumSize(MAX_LOGICAL_REFERENCES)
+            .weakValues()
+            .concurrencyLevel(CACHE_CONCURRENCY_LEVEL)
+            .build();
+
+    this.memoryMonitor = MemoryMonitor.fromOptions(options);
+    this.statusPages = WorkerStatusPages.create(DEFAULT_STATUS_PORT, this.memoryMonitor);
+
+    if (!DataflowRunner.hasExperiment(options, "disable_debug_capture")) {
+      this.debugCaptureManager =
+          initializeAndStartDebugCaptureManager(options, statusPages.getDebugCapturePages());
+    }
+
+    // TODO: this conditional -> two implementations of common interface, or
+    // param/injection
+    if (DataflowRunner.hasExperiment(options, "beam_fn_api")) {
+      Function<MutableNetwork<Node, Edge>, Node> sdkFusedStage =
+          pipeline == null
+              ? RegisterNodeFunction.withoutPipeline(
+                  IdGenerator::generate, sdkHarnessRegistry.beamFnStateApiServiceDescriptor())
+              : RegisterNodeFunction.forPipeline(
+                  pipeline,
+                  IdGenerator::generate,
+                  sdkHarnessRegistry.beamFnStateApiServiceDescriptor());
+      Function<MutableNetwork<Node, Edge>, MutableNetwork<Node, Edge>> lengthPrefixUnknownCoders =
+          LengthPrefixUnknownCoders::forSdkNetwork;
+      Function<MutableNetwork<Node, Edge>, MutableNetwork<Node, Edge>> transformToRunnerNetwork =
+          new CreateRegisterFnOperationFunction(
+              IdGenerator::generate,
+              this::createPortNode,
+              lengthPrefixUnknownCoders.andThen(sdkFusedStage));
+
+      mapTaskToNetwork =
+          mapTaskToBaseNetwork
+              .andThen(new ReplacePgbkWithPrecombineFunction())
+              .andThen(new DeduceNodeLocationsFunction())
+              .andThen(new DeduceFlattenLocationsFunction())
+              .andThen(new CloneAmbiguousFlattensFunction())
+              .andThen(transformToRunnerNetwork)
+              .andThen(LengthPrefixUnknownCoders::andReplaceForRunnerNetwork);
+    } else {
+      mapTaskToNetwork = mapTaskToBaseNetwork;
+    }
+
+    this.memoryMonitorThread = startMemoryMonitorThread(memoryMonitor);
+
+    ExecutionStateSampler.instance().start();
+  }
+
+  private static DebugCapture.Manager initializeAndStartDebugCaptureManager(
+      DataflowWorkerHarnessOptions options, List<DebugCapture.Capturable> debugCapturePages) {
+    DebugCapture.Manager result = new DebugCapture.Manager(options, debugCapturePages);
+    result.start();
+    return result;
+  }
+
+  private static Thread startMemoryMonitorThread(MemoryMonitor memoryMonitor) {
+    Thread result = new Thread(memoryMonitor);
+    result.setDaemon(true);
+    result.setPriority(Thread.MIN_PRIORITY);
+    result.setName("MemoryMonitor");
+    result.start();
+    return result;
+  }
+
+  private Node createPortNode(String predecessorId, String successorId) {
+    return RemoteGrpcPortNode.create(
+        RemoteGrpcPort.newBuilder()
+            .setApiServiceDescriptor(sdkHarnessRegistry.beamFnDataApiServiceDescriptor())
+            .build(),
+        IdGenerator.generate(),
+        IdGenerator.generate(),
+        predecessorId,
+        successorId);
+  }
+
+  /**
+   * Gets WorkItem and performs it; returns true if work succeeded otherwise returns false. This
+   * method will continuously attempt to get work until at least one work item is returned or the
+   * service returns an error. We purposely do this because the service intelligently uses hanging
+   * gets to be able to assign work as quickly as possible without the need for an exponential
+   * backoff strategy for the work request calls.
+   *
+   * <p>getAndPerformWork may throw if there is a failure of the WorkUnitClient.
+   */
+  public boolean getAndPerformWork() throws IOException {
+    while (true) {
+      Optional<WorkItem> work = workUnitClient.getWorkItem();
+      if (work.isPresent()) {
+        WorkItemStatusClient statusProvider = new WorkItemStatusClient(workUnitClient, work.get());
+        return doWork(work.get(), statusProvider);
+      }
+    }
+  }
+
+  /**
+   * Performs the given work; returns true if successful.
+   *
+   * @throws IOException Only if the WorkUnitClient fails.
+   */
+  @VisibleForTesting
+  boolean doWork(WorkItem workItem, WorkItemStatusClient workItemStatusClient) throws IOException {
+    LOG.debug("Executing: {}", workItem);
+
+    DataflowWorkExecutor worker = null;
+    SdkWorkerHarness sdkWorkerHarness = sdkHarnessRegistry.getAvailableWorkerAndAssignWork();
+    try {
+      // Populate PipelineOptions with data from work unit.
+      options.setProject(workItem.getProjectId());
+
+      final String stageName;
+      if (workItem.getMapTask() != null) {
+        stageName = workItem.getMapTask().getStageName();
+      } else if (workItem.getSourceOperationTask() != null) {
+        stageName = workItem.getSourceOperationTask().getStageName();
+      } else {
+        throw new RuntimeException("Unknown kind of work item: " + workItem.toString());
+      }
+
+      CounterSet counterSet = new CounterSet();
+      BatchModeExecutionContext executionContext =
+          BatchModeExecutionContext.create(
+              counterSet,
+              sideInputDataCache,
+              sideInputWeakReferenceCache,
+              readerRegistry,
+              options,
+              stageName,
+              String.valueOf(workItem.getId()));
+
+      if (workItem.getMapTask() != null) {
+        MutableNetwork<Node, Edge> network = mapTaskToNetwork.apply(workItem.getMapTask());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Network as Graphviz .dot: {}", Networks.toDot(network));
+        }
+        worker =
+            mapTaskExecutorFactory.create(
+                sdkWorkerHarness.getControlClientHandler(),
+                sdkWorkerHarness.getDataService(),
+                sdkHarnessRegistry.beamFnDataApiServiceDescriptor(),
+                sdkWorkerHarness.getStateService(),
+                network,
+                options,
+                stageName,
+                readerRegistry,
+                sinkRegistry,
+                executionContext,
+                counterSet,
+                IdGenerator::generate);
+      } else if (workItem.getSourceOperationTask() != null) {
+        worker =
+            SourceOperationExecutorFactory.create(
+                options,
+                workItem.getSourceOperationTask(),
+                counterSet,
+                executionContext,
+                stageName);
+      }
+      workItemStatusClient.setWorker(worker, executionContext);
+
+      DataflowWorkProgressUpdater progressUpdater =
+          new DataflowWorkProgressUpdater(workItemStatusClient, workItem, worker);
+      executeWork(worker, progressUpdater);
+      workItemStatusClient.reportSuccess();
+      return true;
+
+    } catch (Throwable e) {
+      workItemStatusClient.reportError(e);
+      return false;
+
+    } finally {
+      if (worker != null) {
+        try {
+          worker.close();
+        } catch (Exception exn) {
+          LOG.warn(
+              "Uncaught exception while closing worker. All work has already committed or "
+                  + "been marked for retry.",
+              exn);
+        }
+      }
+      if (sdkWorkerHarness != null) {
+        sdkHarnessRegistry.completeWork(sdkWorkerHarness);
+      }
+    }
+  }
+
+  /** Executes the work and report progress. For testing only. */
+  void executeWork(DataflowWorkExecutor worker, DataflowWorkProgressUpdater progressUpdater)
+      throws Exception {
+    progressUpdater.startReportingProgress();
+    // Blocks while executing the work.
+    try {
+      worker.execute();
+    } finally {
+      // stopReportingProgress can throw an exception if the final progress
+      // update fails. For correctness, the task must then be marked as failed.
+      progressUpdater.stopReportingProgress();
+    }
+  }
+
+  /** Runs the status server to report worker health on the specified port. */
+  public void startStatusServer() {
+    statusPages.start();
+  }
+
+  /** Cleanup allocated resources. */
+  @Override
+  public void close() {
+    // TODO: Implement proper cleanup logic.
+    if (this.debugCaptureManager != null) {
+      this.debugCaptureManager.stop();
+    }
+
+    this.memoryMonitor.stop();
+    this.statusPages.stop();
+    ExecutionStateSampler.instance().stop();
+
+    long timeoutMilliSec = 5 * 1000;
+    try {
+      this.memoryMonitorThread.join(timeoutMilliSec);
+    } catch (InterruptedException ex) {
+      LOG.warn("Failed to wait for monitor thread to exit. Ex: {}", ex);
+    }
+    if (this.memoryMonitorThread.isAlive()) {
+      LOG.warn("memoryMonitorThread didn't exit. Please, check for potential memory leaks.");
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java
new file mode 100644
index 0000000..7c498cc
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java
@@ -0,0 +1,517 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.Iterables;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.InMemoryStateInternals;
+import org.apache.beam.runners.core.InMemoryTimerInternals;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.runners.core.TimerInternals.TimerData;
+import org.apache.beam.runners.core.metrics.CounterCell;
+import org.apache.beam.runners.core.metrics.MetricUpdates;
+import org.apache.beam.runners.core.metrics.MetricsContainerImpl;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler;
+import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateSampler;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.metrics.MetricName;
+import org.apache.beam.sdk.metrics.MetricsContainer;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WeightedValue;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.joda.time.Instant;
+
+/** {@link DataflowExecutionContext} for use in batch mode. */
+public class BatchModeExecutionContext
+    extends DataflowExecutionContext<BatchModeExecutionContext.StepContext> {
+
+  protected final Cache<?, WeightedValue<?>> dataCache;
+  protected final Cache<?, ?> logicalReferenceCache;
+  protected final PipelineOptions options;
+  protected final ReaderFactory readerFactory;
+  private Object key;
+
+  private final MetricsContainerRegistry<MetricsContainerImpl> containerRegistry;
+
+  // TODO: Move throttle time Metric to a dedicated namespace.
+  protected static final String DATASTORE_THROTTLE_TIME_NAMESPACE =
+      "org.apache.beam.sdk.io.gcp.datastore.DatastoreV1$DatastoreWriterFn";
+
+  private BatchModeExecutionContext(
+      CounterFactory counterFactory,
+      Cache<?, WeightedValue<?>> dataCache,
+      Cache<?, ?> logicalReferenceCache,
+      ReaderFactory readerFactory,
+      PipelineOptions options,
+      DataflowExecutionStateTracker executionStateTracker,
+      ExecutionStateRegistry executionStateRegistry) {
+    super(
+        counterFactory,
+        createMetricsContainerRegistry(),
+        executionStateTracker,
+        executionStateRegistry,
+        Long.MAX_VALUE);
+    this.logicalReferenceCache = logicalReferenceCache;
+    this.readerFactory = readerFactory;
+    this.options = options;
+    this.dataCache = dataCache;
+    this.containerRegistry =
+        (MetricsContainerRegistry<MetricsContainerImpl>) getMetricsContainerRegistry();
+  }
+
+  private static MetricsContainerRegistry<MetricsContainerImpl> createMetricsContainerRegistry() {
+    return new MetricsContainerRegistry<MetricsContainerImpl>() {
+      @Override
+      protected MetricsContainerImpl createContainer(String stepName) {
+        return new MetricsContainerImpl(stepName);
+      }
+    };
+  }
+
+  public static BatchModeExecutionContext forTesting(
+      PipelineOptions options, CounterFactory counterFactory, String stageName) {
+    BatchModeExecutionStateRegistry stateRegistry = new BatchModeExecutionStateRegistry();
+    return new BatchModeExecutionContext(
+        counterFactory,
+        CacheBuilder.newBuilder()
+            .maximumWeight(1_000_000) // weights are in bytes
+            .weigher(Weighers.fixedWeightKeys(8))
+            .softValues()
+            .recordStats()
+            .<Object, WeightedValue<?>>build(),
+        CacheBuilder.newBuilder().weakValues().build(),
+        ReaderRegistry.defaultRegistry(),
+        options,
+        new DataflowExecutionStateTracker(
+            ExecutionStateSampler.newForTest(),
+            stateRegistry.getState(
+                NameContext.forStage(stageName),
+                "other",
+                null,
+                ScopedProfiler.INSTANCE.emptyScope()),
+            counterFactory,
+            options,
+            "test-work-item-id"),
+        stateRegistry);
+  }
+
+  public static BatchModeExecutionContext forTesting(PipelineOptions options, String stageName) {
+    CounterFactory counterFactory = new CounterFactory();
+    return forTesting(options, counterFactory, stageName);
+  }
+
+  /**
+   * A version of {@link DataflowOperationContext.DataflowExecutionState} supporting per-bundle MSEC
+   * counters.
+   */
+  @VisibleForTesting
+  static class BatchModeExecutionState extends DataflowOperationContext.DataflowExecutionState {
+
+    // Must be volatile so that totalMillisInState modifications are atomic. This field is only
+    // written by the sampler thread in takeSample, and read in a separate thread within
+    // extractUpdate.
+    private volatile long totalMillisInState = 0;
+
+    // This thread is only read and written within extractUpdate, which will be called from either
+    // the progress reporting thread or the execution thread when producing the final update.
+    // The value is not used as part of producing the final update, so the fact this is used in
+    // multiple threads does not lead to incorrect results.
+    private long lastReportedMillis = 0;
+
+    public BatchModeExecutionState(
+        NameContext nameContext,
+        String stateName,
+        @Nullable String requestingStepName,
+        @Nullable Integer inputIndex,
+        @Nullable MetricsContainer metricsContainer,
+        ProfileScope profileScope) {
+      super(nameContext, stateName, requestingStepName, inputIndex, metricsContainer, profileScope);
+    }
+
+    /**
+     * Take sample is only called by the ExecutionStateSampler thread and it is the only place that
+     * writes to totalMillisInState. Thus, we don't need to synchronize these writes, but we just
+     * need to make sure that the value seen by readers (which are in the reporting thread) is
+     * consistent. This is ensured since totalMillisInState is volatile, and thus is atomic.
+     */
+    @SuppressWarnings("NonAtomicVolatileUpdate") // Single writer.
+    @Override
+    public void takeSample(long millisSinceLastSample) {
+      totalMillisInState += millisSinceLastSample;
+    }
+
+    /**
+     * Extract updates in the from of a {@link CounterUpdate}.
+     *
+     * <p>Non-final updates are extracted by the progress reporting thread (single reader, different
+     * from the sampler thread calling takeSample).
+     *
+     * <p>Final updates are extracted by the execution thread, and will be reported after all
+     * processing has completed and the writer thread has been shutdown.
+     */
+    @Nullable
+    @Override
+    public CounterUpdate extractUpdate(boolean isFinalUpdate) {
+      long millisToReport = totalMillisInState;
+      if (millisToReport == lastReportedMillis && !isFinalUpdate) {
+        return null;
+      }
+
+      lastReportedMillis = millisToReport;
+      return createUpdate(true, totalMillisInState);
+    }
+  }
+
+  /** {@link ExecutionStateRegistry} that creates {@link BatchModeExecutionState} instances. */
+  @VisibleForTesting
+  public static class BatchModeExecutionStateRegistry extends ExecutionStateRegistry {
+
+    @Override
+    protected DataflowOperationContext.DataflowExecutionState createState(
+        NameContext nameContext,
+        String stateName,
+        @Nullable String requestingStepName,
+        @Nullable Integer inputIndex,
+        MetricsContainer container,
+        ProfileScope profileScope) {
+      return new BatchModeExecutionState(
+          nameContext, stateName, requestingStepName, inputIndex, container, profileScope);
+    }
+  }
+
+  public static BatchModeExecutionContext create(
+      CounterFactory counterFactory,
+      Cache<?, WeightedValue<?>> dataCache,
+      Cache<?, ?> logicalReferenceCache,
+      ReaderFactory readerFactory,
+      DataflowPipelineOptions options,
+      String stageName,
+      String workItemId) {
+    BatchModeExecutionStateRegistry executionStateRegistry = new BatchModeExecutionStateRegistry();
+    return new BatchModeExecutionContext(
+        counterFactory,
+        dataCache,
+        logicalReferenceCache,
+        readerFactory,
+        options,
+        new DataflowExecutionStateTracker(
+            ExecutionStateSampler.instance(),
+            executionStateRegistry.getState(
+                NameContext.forStage(stageName),
+                "other",
+                null,
+                ScopedProfiler.INSTANCE.emptyScope()),
+            counterFactory,
+            options,
+            workItemId),
+        executionStateRegistry);
+  }
+
+  /** Create a new {@link StepContext}. */
+  @Override
+  protected StepContext createStepContext(DataflowOperationContext operationContext) {
+    return new StepContext(operationContext);
+  }
+
+  /** Sets the key of the work currently being processed. */
+  public void setKey(Object key) {
+    if (!Objects.equals(key, this.key)) {
+      switchStateKey(key);
+    }
+
+    this.key = key;
+  }
+
+  /** @param newKey the key being switched to */
+  protected void switchStateKey(Object newKey) {
+    for (StepContext stepContext : getAllStepContexts()) {
+      stepContext.setKey(newKey);
+    }
+  }
+
+  /**
+   * Returns the key of the work currently being processed.
+   *
+   * <p>If there is not a currently defined key, returns null.
+   */
+  @Nullable
+  public Object getKey() {
+    return key;
+  }
+
+  @Override
+  protected SideInputReader getSideInputReader(
+      Iterable<? extends SideInputInfo> sideInputInfos, DataflowOperationContext operationContext)
+      throws Exception {
+    return new LazilyInitializedSideInputReader(
+        sideInputInfos,
+        () -> {
+          try {
+            return IsmSideInputReader.of(
+                sideInputInfos,
+                options,
+                BatchModeExecutionContext.this,
+                readerFactory,
+                operationContext);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+        });
+  }
+
+  @Override
+  protected SideInputReader getSideInputReaderForViews(
+      Iterable<? extends PCollectionView<?>> sideInputViews) {
+    throw new UnsupportedOperationException(
+        "Cannot call getSideInputReaderForViews for batch DataflowWorker: "
+            + "the MapTask specification should have had SideInputInfo descriptors "
+            + "for each side input, and a SideInputReader provided via getSideInputReader");
+  }
+
+  // TODO: Expose a keyed sub-cache which allows one to store all cached values in their
+  // own namespace.
+  public <K, V> Cache<K, V> getDataCache() {
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    Cache<K, V> rval = (Cache) dataCache;
+    return rval;
+  }
+
+  public <K, V> Cache<K, V> getLogicalReferenceCache() {
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    Cache<K, V> rval = (Cache) logicalReferenceCache;
+    return rval;
+  }
+
+  /** {@link DataflowStepContext} used in batch mode. */
+  public class StepContext extends DataflowExecutionContext.DataflowStepContext {
+
+    // State internals only for use by the system, lazily instantiated
+    @Nullable private InMemoryStateInternals<Object> systemStateInternals;
+
+    // State internals scoped to the user, lazily instantiated
+    @Nullable private InMemoryStateInternals<Object> userStateInternals;
+
+    // Timer internals scoped to the user, lazily instantiated
+    @Nullable private InMemoryTimerInternals userTimerInternals;
+
+    private InMemoryStateInternals<Object> getUserStateInternals() {
+      if (userStateInternals == null) {
+        userStateInternals = InMemoryStateInternals.forKey(getKey());
+      }
+      return userStateInternals;
+    }
+
+    private InMemoryTimerInternals getUserTimerInternals() {
+      if (userTimerInternals == null) {
+        userTimerInternals = new InMemoryTimerInternals();
+      }
+      return userTimerInternals;
+    }
+
+    private StepContext(DataflowOperationContext operationContext) {
+      super(operationContext.nameContext());
+      systemStateInternals = null;
+      userStateInternals = null;
+      userTimerInternals = null;
+    }
+
+    public void setKey(Object newKey) {
+      // When the key changes, wipe out existing state and timers for later
+      // lazy instantiation.
+
+      // In batch mode, a specific key is always processed contiguously
+      // because the state is either used after a GroupByKeyOnly where
+      // each key only occurs once, or after some ParDo's that preserved
+      // the key.
+      systemStateInternals = null;
+
+      userStateInternals = null;
+      userTimerInternals = null;
+    }
+
+    @Override
+    public StateInternals stateInternals() {
+      if (systemStateInternals == null) {
+        systemStateInternals = InMemoryStateInternals.forKey(getKey());
+      }
+      return systemStateInternals;
+    }
+
+    @Override
+    public TimerInternals timerInternals() {
+      throw new UnsupportedOperationException(
+          "System timerInternals() are not supported in Batch mode."
+              + " Perhaps you meant stepContext.namespacedToUser().timerInternals()");
+    }
+
+    @Override
+    @Nullable
+    public <W extends BoundedWindow> TimerData getNextFiredTimer(Coder<W> windowCoder) {
+      // There are no actual timer firings, since in batch all state is cleaned up trivially
+      // and all user processing time timers are "expired" by the time they fire.
+      // Event time timers are handled in the UserStepContext
+      return null;
+    }
+
+    @Override
+    public <W extends BoundedWindow> void setStateCleanupTimer(
+        String timerId, W window, Coder<W> windowCoder, Instant cleanupTime) {
+      // noop, as the state will be discarded when the step is complete
+    }
+
+    @Override
+    public DataflowStepContext namespacedToUser() {
+      return new UserStepContext(this);
+    }
+  }
+
+  /**
+   * A specialized {@link StepContext} that uses provided {@link StateInternals} and {@link
+   * TimerInternals} for user state and timers.
+   */
+  private static class UserStepContext extends DataflowStepContext {
+
+    private final BatchModeExecutionContext.StepContext wrapped;
+
+    public UserStepContext(BatchModeExecutionContext.StepContext wrapped) {
+      super(wrapped.getNameContext());
+      this.wrapped = wrapped;
+    }
+
+    @Override
+    public StateInternals stateInternals() {
+      return wrapped.getUserStateInternals();
+    }
+
+    @Override
+    public TimerInternals timerInternals() {
+      return wrapped.getUserTimerInternals();
+    }
+
+    @Nullable
+    @Override
+    public <W extends BoundedWindow> TimerData getNextFiredTimer(Coder<W> windowCoder) {
+      // Only event time timers fire, as processing time timers are reserved until after the
+      // bundle is complete, so they are all delivered droppably late
+      //
+      // Note also that timers hold the _output_ watermark, but the input watermark is
+      // advanced to infinity when timers begin to process
+      try {
+        wrapped.getUserTimerInternals().advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
+      } catch (Exception e) {
+        throw new IllegalStateException("Exception thrown advancing watermark", e);
+      }
+
+      return wrapped.getUserTimerInternals().removeNextEventTimer();
+    }
+
+    @Override
+    public <W extends BoundedWindow> void setStateCleanupTimer(
+        String timerId, W window, Coder<W> windowCoder, Instant cleanupTime) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "setStateCleanupTimer should not be called on %s, only on a system %s",
+              getClass().getSimpleName(),
+              BatchModeExecutionContext.StepContext.class.getSimpleName()));
+    }
+
+    @Override
+    public DataflowStepContext namespacedToUser() {
+      return this;
+    }
+  }
+
+  /**
+   * Returns {@link CounterUpdate} protos representing the latest cumulative values of all
+   * user-defined metrics reported within this execution context.
+   */
+  public Iterable<CounterUpdate> extractMetricUpdates(boolean isFinalUpdate) {
+    return containerRegistry
+        .getContainers()
+        .transformAndConcat(
+            container -> {
+              MetricUpdates updates;
+              if (isFinalUpdate) {
+                // getCumulative returns cumulative values for all metrics.
+                updates = container.getCumulative();
+              } else {
+                // getUpdates returns cumulative values only for metrics which have changes
+                // since the last time updates were committed.
+                updates = container.getUpdates();
+              }
+              return Iterables.concat(
+                  FluentIterable.from(updates.counterUpdates())
+                      .transform(
+                          (update) ->
+                              MetricsToCounterUpdateConverter.fromCounter(
+                                  update.getKey(), true, update.getUpdate())),
+                  FluentIterable.from(updates.distributionUpdates())
+                      .transform(
+                          (update) ->
+                              MetricsToCounterUpdateConverter.fromDistribution(
+                                  update.getKey(), true, update.getUpdate())));
+            });
+  }
+
+  public void commitMetricUpdates() {
+    for (MetricsContainerImpl container : containerRegistry.getContainers()) {
+      container.commitUpdates();
+    }
+  }
+
+  public Iterable<CounterUpdate> extractMsecCounters(boolean isFinalUpdate) {
+    return executionStateRegistry.extractUpdates(isFinalUpdate);
+  }
+
+  public Long extractThrottleTime() {
+    Long totalThrottleTime = 0L;
+    for (MetricsContainerImpl container : containerRegistry.getContainers()) {
+      // TODO: Update Datastore to use generic throttling-msecs metric.
+      CounterCell throttleTime =
+          container.tryGetCounter(
+              MetricName.named(
+                  BatchModeExecutionContext.DATASTORE_THROTTLE_TIME_NAMESPACE,
+                  "cumulativeThrottlingSeconds"));
+      if (throttleTime != null) {
+        totalThrottleTime += throttleTime.getCumulative();
+      }
+      CounterCell throttlingMsecs =
+          container.tryGetCounter(DataflowSystemMetrics.THROTTLING_MSECS_METRIC_NAME);
+      if (throttlingMsecs != null) {
+        totalThrottleTime += TimeUnit.MICROSECONDS.toSeconds(throttlingMsecs.getCumulative());
+      }
+    }
+    return totalThrottleTime;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java
new file mode 100644
index 0000000..8db9ca9
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java
@@ -0,0 +1,83 @@
+/*
+ * 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.dataflow.worker;
+
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.joda.time.Instant;
+
+/**
+ * A {@link ParDoFn} over per-key iterables that applies an underlying {@link ParDoFn} to the
+ * elements of an underlying iterable, then processes its timers.
+ *
+ * <p>Each input element must be a {@link KV} where the value is an iterable of {@link WindowedValue
+ * WindowedValues}.
+ */
+class BatchModeUngroupingParDoFn<K, V> implements ParDoFn {
+
+  private final ParDoFn underlyingParDoFn;
+  private final BatchModeExecutionContext.StepContext stepContext;
+
+  BatchModeUngroupingParDoFn(
+      BatchModeExecutionContext.StepContext stepContext, ParDoFn underlyingParDoFn) {
+    this.underlyingParDoFn = underlyingParDoFn;
+    this.stepContext = stepContext;
+  }
+
+  @Override
+  public void startBundle(Receiver... receivers) throws Exception {
+    underlyingParDoFn.startBundle(receivers);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void processElement(Object untypedElem) throws Exception {
+    WindowedValue<?> windowedValue = (WindowedValue<?>) untypedElem;
+
+    KV<K, Iterable<KV<Instant, WindowedValue<V>>>> gbkElem =
+        (KV<K, Iterable<KV<Instant, WindowedValue<V>>>>) windowedValue.getValue();
+
+    // Each GBK output is the beginning of a key
+    stepContext.setKey(gbkElem.getKey());
+
+    for (KV<Instant, WindowedValue<V>> timestampedElem : gbkElem.getValue()) {
+      underlyingParDoFn.processElement(timestampedElem.getValue());
+    }
+
+    // Process all the timers for the key, since the watermark is moved to infinity
+    underlyingParDoFn.processTimers();
+  }
+
+  @Override
+  public void processTimers() throws Exception {
+    // The timers for the underlying ParDoFn are processed at the end of each element
+  }
+
+  @Override
+  public void finishBundle() throws Exception {
+    underlyingParDoFn.finishBundle();
+  }
+
+  @Override
+  public void abort() throws Exception {
+    underlyingParDoFn.abort();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java
new file mode 100644
index 0000000..39a7902
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java
@@ -0,0 +1,653 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.client.util.Throwables;
+import com.google.api.services.dataflow.model.InstructionOutput;
+import com.google.api.services.dataflow.model.MapTask;
+import com.google.api.services.dataflow.model.MultiOutputInfo;
+import com.google.api.services.dataflow.model.ParDoInstruction;
+import com.google.api.services.dataflow.model.ParallelInstruction;
+import com.google.api.services.dataflow.model.PartialGroupByKeyInstruction;
+import com.google.api.services.dataflow.model.ReadInstruction;
+import com.google.api.services.dataflow.model.Source;
+import com.google.api.services.dataflow.model.WriteInstruction;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableTable;
+import com.google.common.collect.Iterables;
+import com.google.common.graph.MutableNetwork;
+import com.google.common.graph.Network;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Target;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.runners.core.ElementByteSizeObservable;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.CloudObjects;
+import org.apache.beam.runners.dataflow.worker.DataflowExecutionContext.DataflowStepContext;
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.fn.control.BeamFnMapTaskExecutor;
+import org.apache.beam.runners.dataflow.worker.fn.control.RegisterAndProcessBundleOperation;
+import org.apache.beam.runners.dataflow.worker.fn.data.RemoteGrpcPortReadOperation;
+import org.apache.beam.runners.dataflow.worker.fn.data.RemoteGrpcPortWriteOperation;
+import org.apache.beam.runners.dataflow.worker.graph.Edges.Edge;
+import org.apache.beam.runners.dataflow.worker.graph.Edges.MultiOutputInfoEdge;
+import org.apache.beam.runners.dataflow.worker.graph.Networks;
+import org.apache.beam.runners.dataflow.worker.graph.Networks.TypeSafeNodeFunction;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.FetchAndFilterStreamingSideInputsNode;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.InstructionOutputNode;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.Node;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.OperationNode;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.OutputReceiverNode;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.ParallelInstructionNode;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.RegisterRequestNode;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.RemoteGrpcPortNode;
+import org.apache.beam.runners.dataflow.worker.util.CloudSourceUtils;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.FlattenOperation;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Operation;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OperationContext;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputReceiver;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoOperation;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Sink;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.WriteOperation;
+import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
+import org.apache.beam.runners.fnexecution.data.FnDataService;
+import org.apache.beam.runners.fnexecution.state.StateDelegator;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Creates a {@link DataflowMapTaskExecutor} from a {@link MapTask} definition. */
+public class BeamFnMapTaskExecutorFactory implements DataflowMapTaskExecutorFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamFnMapTaskExecutorFactory.class);
+
+  public static BeamFnMapTaskExecutorFactory defaultFactory() {
+    return new BeamFnMapTaskExecutorFactory();
+  }
+
+  private BeamFnMapTaskExecutorFactory() {}
+
+  /**
+   * Creates a new {@link DataflowMapTaskExecutor} from the given {@link MapTask} definition using
+   * the provided {@link ReaderFactory}.
+   */
+  @Override
+  public DataflowMapTaskExecutor create(
+      InstructionRequestHandler instructionRequestHandler,
+      FnDataService beamFnDataService,
+      Endpoints.ApiServiceDescriptor dataApiServiceDescriptor,
+      StateDelegator beamFnStateDelegator,
+      MutableNetwork<Node, Edge> network,
+      PipelineOptions options,
+      String stageName,
+      ReaderFactory readerFactory,
+      SinkFactory sinkFactory,
+      DataflowExecutionContext<?> executionContext,
+      CounterSet counterSet,
+      Supplier<String> idGenerator) {
+
+    // TODO: remove this once we trust the code paths
+    checkArgument(
+        DataflowRunner.hasExperiment(options.as(DataflowPipelineDebugOptions.class), "beam_fn_api"),
+        "%s should only be used when beam_fn_api is enabled",
+        getClass().getSimpleName());
+
+    // Swap out all the InstructionOutput nodes with OutputReceiver nodes
+    Networks.replaceDirectedNetworkNodes(
+        network, createOutputReceiversTransform(stageName, counterSet));
+
+    // Swap out all the RegisterFnRequest nodes with Operation nodes
+    Networks.replaceDirectedNetworkNodes(
+        network,
+        createOperationTransformForRegisterFnNodes(
+            idGenerator,
+            instructionRequestHandler,
+            beamFnStateDelegator,
+            stageName,
+            executionContext));
+
+    // Swap out all the RemoteGrpcPort nodes with Operation nodes, note that it is expected
+    // that the RegisterFnRequest nodes have already been replaced.
+    Networks.replaceDirectedNetworkNodes(
+        network,
+        createOperationTransformForGrpcPortNodes(
+            network,
+            beamFnDataService,
+            // TODO: Set NameContext properly for these operations.
+            executionContext.createOperationContext(
+                NameContext.create(stageName, stageName, stageName, stageName))));
+
+    // Swap out all the FetchAndFilterStreamingSideInput nodes with operation nodes
+    Networks.replaceDirectedNetworkNodes(
+        network,
+        createOperationTransformForFetchAndFilterStreamingSideInputNodes(
+            network,
+            idGenerator,
+            instructionRequestHandler,
+            beamFnDataService,
+            dataApiServiceDescriptor,
+            executionContext,
+            // TODO: Set NameContext properly for these operations.
+            executionContext.createOperationContext(
+                NameContext.create(stageName, stageName, stageName, stageName))));
+
+    // Swap out all the ParallelInstruction nodes with Operation nodes
+    Networks.replaceDirectedNetworkNodes(
+        network,
+        createOperationTransformForParallelInstructionNodes(
+            stageName, network, options, readerFactory, sinkFactory, executionContext));
+
+    // Collect all the operations within the network and attach all the operations as receivers
+    // to preceding output receivers.
+    List<Operation> topoSortedOperations = new ArrayList<>();
+    for (OperationNode node :
+        Iterables.filter(Networks.topologicalOrder(network), OperationNode.class)) {
+      topoSortedOperations.add(node.getOperation());
+
+      for (Node predecessor :
+          Iterables.filter(network.predecessors(node), OutputReceiverNode.class)) {
+        ((OutputReceiverNode) predecessor)
+            .getOutputReceiver()
+            .addOutput((Receiver) node.getOperation());
+      }
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.info("Map task network: {}", Networks.toDot(network));
+    }
+
+    return BeamFnMapTaskExecutor.withSharedCounterSet(
+        topoSortedOperations, counterSet, executionContext.getExecutionStateTracker());
+  }
+
+  private Function<Node, Node> createOperationTransformForFetchAndFilterStreamingSideInputNodes(
+      MutableNetwork<Node, Edge> network,
+      Supplier<String> idGenerator,
+      InstructionRequestHandler instructionRequestHandler,
+      FnDataService beamFnDataService,
+      Endpoints.ApiServiceDescriptor dataApiServiceDescriptor,
+      DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext) {
+    return new TypeSafeNodeFunction<FetchAndFilterStreamingSideInputsNode>(
+        FetchAndFilterStreamingSideInputsNode.class) {
+
+      @Override
+      public Node typedApply(FetchAndFilterStreamingSideInputsNode input) {
+        OutputReceiverNode output =
+            (OutputReceiverNode) Iterables.getOnlyElement(network.successors(input));
+        return OperationNode.create(
+            new FetchAndFilterStreamingSideInputsOperation<>(
+                new OutputReceiver[] {output.getOutputReceiver()},
+                operationContext,
+                instructionRequestHandler,
+                beamFnDataService,
+                dataApiServiceDescriptor,
+                idGenerator,
+                (Coder<WindowedValue<Object>>) output.getCoder(),
+                (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy(),
+                executionContext.getStepContext(operationContext),
+                input.getPCollectionViewsToWindowMappingFns()));
+      }
+    };
+  }
+
+  private Function<Node, Node> createOperationTransformForGrpcPortNodes(
+      final Network<Node, Edge> network,
+      final FnDataService beamFnDataService,
+      final OperationContext context) {
+    return new TypeSafeNodeFunction<RemoteGrpcPortNode>(RemoteGrpcPortNode.class) {
+      @Override
+      public Node typedApply(RemoteGrpcPortNode input) {
+        RegisterAndProcessBundleOperation registerFnOperation =
+            (RegisterAndProcessBundleOperation)
+                Iterables.getOnlyElement(
+                        Iterables.filter(network.adjacentNodes(input), OperationNode.class))
+                    .getOperation();
+
+        // The coder comes from the one and only adjacent output node
+        Coder<?> coder =
+            Iterables.getOnlyElement(
+                    Iterables.filter(network.adjacentNodes(input), OutputReceiverNode.class))
+                .getCoder();
+        // We figure out whether we are outputting some where if the output node is a
+        // successor.
+        Iterable<OutputReceiverNode> outputReceiverNodes =
+            Iterables.filter(network.successors(input), OutputReceiverNode.class);
+        Operation operation;
+        if (outputReceiverNodes.iterator().hasNext()) {
+          Target target =
+              Target.newBuilder()
+                  .setPrimitiveTransformReference(input.getPrimitiveTransformId())
+                  .setName(input.getOutputId())
+                  .build();
+          OutputReceiver[] outputReceivers =
+              new OutputReceiver[] {
+                Iterables.getOnlyElement(outputReceiverNodes).getOutputReceiver()
+              };
+
+          operation =
+              new RemoteGrpcPortReadOperation<>(
+                  beamFnDataService,
+                  target,
+                  registerFnOperation::getProcessBundleInstructionId,
+                  (Coder) coder,
+                  outputReceivers,
+                  context);
+        } else {
+          Target target =
+              Target.newBuilder()
+                  .setPrimitiveTransformReference(input.getPrimitiveTransformId())
+                  .setName(input.getInputId())
+                  .build();
+
+          operation =
+              new RemoteGrpcPortWriteOperation<>(
+                  beamFnDataService,
+                  target,
+                  registerFnOperation::getProcessBundleInstructionId,
+                  (Coder) coder,
+                  context);
+        }
+        return OperationNode.create(operation);
+      }
+    };
+  }
+
+  private Function<Node, Node> createOperationTransformForRegisterFnNodes(
+      final Supplier<String> idGenerator,
+      final InstructionRequestHandler instructionRequestHandler,
+      final StateDelegator beamFnStateDelegator,
+      final String stageName,
+      final DataflowExecutionContext<?> executionContext) {
+    return new TypeSafeNodeFunction<RegisterRequestNode>(RegisterRequestNode.class) {
+      @Override
+      public Node typedApply(RegisterRequestNode input) {
+        ImmutableMap.Builder<String, DataflowOperationContext>
+            ptransformIdToOperationContextBuilder = ImmutableMap.builder();
+        ImmutableMap.Builder<String, DataflowStepContext> ptransformIdToStepContext =
+            ImmutableMap.builder();
+        for (Map.Entry<String, NameContext> entry :
+            input.getPTransformIdToPartialNameContextMap().entrySet()) {
+          NameContext fullNameContext =
+              NameContext.create(
+                  stageName,
+                  entry.getValue().originalName(),
+                  entry.getValue().systemName(),
+                  entry.getValue().userName());
+
+          DataflowOperationContext operationContext =
+              executionContext.createOperationContext(fullNameContext);
+          ptransformIdToOperationContextBuilder.put(entry.getKey(), operationContext);
+          ptransformIdToStepContext.put(
+              entry.getKey(), executionContext.getStepContext(operationContext));
+        }
+
+        ImmutableMap<String, DataflowOperationContext> ptransformIdToOperationContexts =
+            ptransformIdToOperationContextBuilder.build();
+
+        ImmutableMap<String, SideInputReader> ptransformIdToSideInputReaders =
+            buildPTransformIdToSideInputReadersMap(
+                executionContext, input, ptransformIdToOperationContexts);
+
+        ImmutableTable<String, String, PCollectionView<?>>
+            ptransformIdToSideInputIdToPCollectionView =
+                buildPTransformIdToSideInputIdToPCollectionView(input);
+
+        return OperationNode.create(
+            new RegisterAndProcessBundleOperation(
+                idGenerator,
+                instructionRequestHandler,
+                beamFnStateDelegator,
+                input.getRegisterRequest(),
+                ptransformIdToOperationContexts,
+                ptransformIdToStepContext.build(),
+                ptransformIdToSideInputReaders,
+                ptransformIdToSideInputIdToPCollectionView,
+                // TODO: Set NameContext properly for these operations.
+                executionContext.createOperationContext(
+                    NameContext.create(stageName, stageName, stageName, stageName))));
+      }
+    };
+  }
+
+  /** Returns a map from PTransform id to side input reader. */
+  private static ImmutableMap<String, SideInputReader> buildPTransformIdToSideInputReadersMap(
+      DataflowExecutionContext executionContext,
+      RegisterRequestNode registerRequestNode,
+      ImmutableMap<String, DataflowOperationContext> ptransformIdToOperationContexts) {
+
+    ImmutableMap.Builder<String, SideInputReader> ptransformIdToSideInputReaders =
+        ImmutableMap.builder();
+    for (Map.Entry<String, Iterable<PCollectionView<?>>> ptransformIdToPCollectionView :
+        registerRequestNode.getPTransformIdToPCollectionViewMap().entrySet()) {
+      try {
+        ptransformIdToSideInputReaders.put(
+            ptransformIdToPCollectionView.getKey(),
+            executionContext.getSideInputReader(
+                // Note that the side input infos will only be populated for a batch pipeline
+                registerRequestNode
+                    .getPTransformIdToSideInputInfoMap()
+                    .get(ptransformIdToPCollectionView.getKey()),
+                ptransformIdToPCollectionView.getValue(),
+                ptransformIdToOperationContexts.get(ptransformIdToPCollectionView.getKey())));
+      } catch (Exception e) {
+        throw Throwables.propagate(e);
+      }
+    }
+    return ptransformIdToSideInputReaders.build();
+  }
+
+  /**
+   * Returns a table where the row key is the PTransform id, the column key is the side input id,
+   * and the value is the corresponding PCollectionView.
+   */
+  private static ImmutableTable<String, String, PCollectionView<?>>
+      buildPTransformIdToSideInputIdToPCollectionView(RegisterRequestNode registerRequestNode) {
+    ImmutableTable.Builder<String, String, PCollectionView<?>>
+        ptransformIdToSideInputIdToPCollectionViewBuilder = ImmutableTable.builder();
+    for (Map.Entry<String, Iterable<PCollectionView<?>>> ptransformIdToPCollectionViews :
+        registerRequestNode.getPTransformIdToPCollectionViewMap().entrySet()) {
+      for (PCollectionView<?> pCollectionView : ptransformIdToPCollectionViews.getValue()) {
+        ptransformIdToSideInputIdToPCollectionViewBuilder.put(
+            ptransformIdToPCollectionViews.getKey(),
+            pCollectionView.getTagInternal().getId(),
+            pCollectionView);
+      }
+    }
+
+    return ptransformIdToSideInputIdToPCollectionViewBuilder.build();
+  }
+
+  /**
+   * Creates an {@link Operation} from the given {@link ParallelInstruction} definition using the
+   * provided {@link ReaderFactory}.
+   */
+  Function<Node, Node> createOperationTransformForParallelInstructionNodes(
+      final String stageName,
+      final Network<Node, Edge> network,
+      final PipelineOptions options,
+      final ReaderFactory readerFactory,
+      final SinkFactory sinkFactory,
+      final DataflowExecutionContext<?> executionContext) {
+
+    return new TypeSafeNodeFunction<ParallelInstructionNode>(ParallelInstructionNode.class) {
+      @Override
+      public Node typedApply(ParallelInstructionNode node) {
+        ParallelInstruction instruction = node.getParallelInstruction();
+        NameContext nameContext =
+            NameContext.create(
+                stageName,
+                instruction.getOriginalName(),
+                instruction.getSystemName(),
+                instruction.getName());
+        try {
+          DataflowOperationContext context = executionContext.createOperationContext(nameContext);
+          if (instruction.getRead() != null) {
+            return createReadOperation(
+                network, node, options, readerFactory, executionContext, context);
+          } else if (instruction.getWrite() != null) {
+            return createWriteOperation(node, options, sinkFactory, executionContext, context);
+          } else if (instruction.getParDo() != null) {
+            return createParDoOperation(network, node, options, executionContext, context);
+          } else if (instruction.getPartialGroupByKey() != null) {
+            return createPartialGroupByKeyOperation(
+                network, node, options, executionContext, context);
+          } else if (instruction.getFlatten() != null) {
+            return createFlattenOperation(network, node, context);
+          } else {
+            throw new IllegalArgumentException(
+                String.format("Unexpected instruction: %s", instruction));
+          }
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+
+  OperationNode createReadOperation(
+      Network<Node, Edge> network,
+      ParallelInstructionNode node,
+      PipelineOptions options,
+      ReaderFactory readerFactory,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+
+    ParallelInstruction instruction = node.getParallelInstruction();
+    ReadInstruction read = instruction.getRead();
+
+    Source cloudSource = CloudSourceUtils.flattenBaseSpecs(read.getSource());
+    CloudObject sourceSpec = CloudObject.fromSpec(cloudSource.getSpec());
+    Coder<?> coder =
+        CloudObjects.coderFromCloudObject(CloudObject.fromSpec(cloudSource.getCodec()));
+    NativeReader<?> reader =
+        readerFactory.create(sourceSpec, coder, options, executionContext, operationContext);
+    OutputReceiver[] receivers = getOutputReceivers(network, node);
+    return OperationNode.create(ReadOperation.create(reader, receivers, operationContext));
+  }
+
+  OperationNode createWriteOperation(
+      ParallelInstructionNode node,
+      PipelineOptions options,
+      SinkFactory sinkFactory,
+      DataflowExecutionContext executionContext,
+      DataflowOperationContext context)
+      throws Exception {
+
+    ParallelInstruction instruction = node.getParallelInstruction();
+    WriteInstruction write = instruction.getWrite();
+    Coder<?> coder =
+        CloudObjects.coderFromCloudObject(CloudObject.fromSpec(write.getSink().getCodec()));
+    CloudObject cloudSink = CloudObject.fromSpec(write.getSink().getSpec());
+    Sink<?> sink = sinkFactory.create(cloudSink, coder, options, executionContext, context);
+    return OperationNode.create(WriteOperation.create(sink, EMPTY_OUTPUT_RECEIVER_ARRAY, context));
+  }
+
+  private ParDoFnFactory parDoFnFactory = new DefaultParDoFnFactory();
+
+  private OperationNode createParDoOperation(
+      Network<Node, Edge> network,
+      ParallelInstructionNode node,
+      PipelineOptions options,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+
+    ParallelInstruction instruction = node.getParallelInstruction();
+    ParDoInstruction parDo = instruction.getParDo();
+
+    TupleTag<?> mainOutputTag = tupleTag(parDo.getMultiOutputInfos().get(0));
+    ImmutableMap.Builder<TupleTag<?>, Integer> outputTagsToReceiverIndicesBuilder =
+        ImmutableMap.builder();
+    int successorOffset = 0;
+    for (Node successor : network.successors(node)) {
+      for (Edge edge : network.edgesConnecting(node, successor)) {
+        outputTagsToReceiverIndicesBuilder.put(
+            tupleTag(((MultiOutputInfoEdge) edge).getMultiOutputInfo()), successorOffset);
+      }
+      successorOffset += 1;
+    }
+    ParDoFn fn =
+        parDoFnFactory.create(
+            options,
+            CloudObject.fromSpec(parDo.getUserFn()),
+            parDo.getSideInputs(),
+            mainOutputTag,
+            outputTagsToReceiverIndicesBuilder.build(),
+            executionContext,
+            operationContext);
+
+    OutputReceiver[] receivers = getOutputReceivers(network, node);
+    return OperationNode.create(new ParDoOperation(fn, receivers, operationContext));
+  }
+
+  private static <V> TupleTag<V> tupleTag(MultiOutputInfo multiOutputInfo) {
+    return new TupleTag<>(multiOutputInfo.getTag());
+  }
+
+  <K> OperationNode createPartialGroupByKeyOperation(
+      Network<Node, Edge> network,
+      ParallelInstructionNode node,
+      PipelineOptions options,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+
+    ParallelInstruction instruction = node.getParallelInstruction();
+    PartialGroupByKeyInstruction pgbk = instruction.getPartialGroupByKey();
+    OutputReceiver[] receivers = getOutputReceivers(network, node);
+
+    Coder<?> windowedCoder =
+        CloudObjects.coderFromCloudObject(CloudObject.fromSpec(pgbk.getInputElementCodec()));
+    if (!(windowedCoder instanceof WindowedValueCoder)) {
+      throw new IllegalArgumentException(
+          String.format(
+              "unexpected kind of input coder for PartialGroupByKeyOperation: %s", windowedCoder));
+    }
+    Coder<?> elemCoder = ((WindowedValueCoder<?>) windowedCoder).getValueCoder();
+    if (!(elemCoder instanceof KvCoder)) {
+      throw new IllegalArgumentException(
+          String.format(
+              "unexpected kind of input element coder for PartialGroupByKeyOperation: %s",
+              elemCoder));
+    }
+
+    @SuppressWarnings("unchecked")
+    KvCoder<K, ?> keyedElementCoder = (KvCoder<K, ?>) elemCoder;
+
+    CloudObject cloudUserFn =
+        pgbk.getValueCombiningFn() != null
+            ? CloudObject.fromSpec(pgbk.getValueCombiningFn())
+            : null;
+    ParDoFn fn =
+        PartialGroupByKeyParDoFns.create(
+            options,
+            keyedElementCoder,
+            cloudUserFn,
+            pgbk.getSideInputs(),
+            Arrays.<Receiver>asList(receivers),
+            executionContext,
+            operationContext);
+
+    return OperationNode.create(new ParDoOperation(fn, receivers, operationContext));
+  }
+
+  OperationNode createFlattenOperation(
+      Network<Node, Edge> network, ParallelInstructionNode node, OperationContext context) {
+    OutputReceiver[] receivers = getOutputReceivers(network, node);
+    return OperationNode.create(new FlattenOperation(receivers, context));
+  }
+
+  /**
+   * Returns a function which can convert {@link InstructionOutput}s into {@link OutputReceiver}s.
+   */
+  static Function<Node, Node> createOutputReceiversTransform(
+      final String stageName, final CounterFactory counterFactory) {
+    return new TypeSafeNodeFunction<InstructionOutputNode>(InstructionOutputNode.class) {
+      @Override
+      public Node typedApply(InstructionOutputNode input) {
+        InstructionOutput cloudOutput = input.getInstructionOutput();
+        OutputReceiver outputReceiver = new OutputReceiver();
+        Coder<?> coder =
+            CloudObjects.coderFromCloudObject(CloudObject.fromSpec(cloudOutput.getCodec()));
+
+        @SuppressWarnings("unchecked")
+        ElementCounter outputCounter =
+            new DataflowOutputCounter(
+                cloudOutput.getName(),
+                new ElementByteSizeObservableCoder<>(coder),
+                counterFactory,
+                NameContext.create(
+                    stageName,
+                    cloudOutput.getOriginalName(),
+                    cloudOutput.getSystemName(),
+                    cloudOutput.getName()));
+        outputReceiver.addOutputCounter(outputCounter);
+
+        return OutputReceiverNode.create(outputReceiver, coder);
+      }
+    };
+  }
+
+  private static final OutputReceiver[] EMPTY_OUTPUT_RECEIVER_ARRAY = new OutputReceiver[0];
+
+  static OutputReceiver[] getOutputReceivers(Network<Node, Edge> network, Node node) {
+    int outDegree = network.outDegree(node);
+    if (outDegree == 0) {
+      return EMPTY_OUTPUT_RECEIVER_ARRAY;
+    }
+
+    OutputReceiver[] receivers = new OutputReceiver[outDegree];
+    Iterator<Node> receiverNodes = network.successors(node).iterator();
+    int i = 0;
+    do {
+      receivers[i] = ((OutputReceiverNode) receiverNodes.next()).getOutputReceiver();
+      i += 1;
+    } while (receiverNodes.hasNext());
+
+    return receivers;
+  }
+
+  /** Adapts a Coder to the ElementByteSizeObservable interface. */
+  public static class ElementByteSizeObservableCoder<T> implements ElementByteSizeObservable<T> {
+    final Coder<T> coder;
+
+    public ElementByteSizeObservableCoder(Coder<T> coder) {
+      this.coder = coder;
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(T value) {
+      return coder.isRegisterByteSizeObserverCheap(value);
+    }
+
+    @Override
+    public void registerByteSizeObserver(T value, ElementByteSizeObserver observer)
+        throws Exception {
+      coder.registerByteSizeObserver(value, observer);
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java
new file mode 100644
index 0000000..bb8fc59
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java
@@ -0,0 +1,53 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.io.ByteStreams;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.vendor.protobuf.v3.com.google.protobuf.ByteString;
+
+/**
+ * A simplified {@link Coder} for {@link ByteString}, to avoid a dependency on
+ * beam-java-sdk-extensions-protobuf.
+ */
+public class ByteStringCoder extends AtomicCoder<ByteString> {
+  public static ByteStringCoder of() {
+    return INSTANCE;
+  }
+
+  private static final ByteStringCoder INSTANCE = new ByteStringCoder();
+
+  private ByteStringCoder() {}
+
+  @Override
+  public void encode(ByteString value, OutputStream os) throws IOException {
+    VarInt.encode(value.size(), os);
+    value.writeTo(os);
+  }
+
+  @Override
+  public ByteString decode(InputStream is) throws IOException {
+    int size = VarInt.decodeInt(is);
+    return ByteString.readFrom(ByteStreams.limit(is, size), size);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java
new file mode 100644
index 0000000..2586c60
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java
@@ -0,0 +1,103 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.io.ByteStreams;
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ByteArrayShufflePosition;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker.ExecutionState;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleBatchReader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleEntry;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ShufflePosition;
+
+/** ChunkingShuffleBatchReader reads data from a shuffle dataset using a ShuffleReader. */
+final class ChunkingShuffleBatchReader implements ShuffleBatchReader {
+  private ShuffleReader reader;
+  private ExecutionStateTracker tracker;
+  private ExecutionState readState;
+
+  public ChunkingShuffleBatchReader(
+      BatchModeExecutionContext executionContext,
+      DataflowOperationContext operationContext,
+      ShuffleReader reader) {
+    this.reader = reader;
+    this.readState = operationContext.newExecutionState("read-shuffle");
+    this.tracker = executionContext.getExecutionStateTracker();
+  }
+
+  @Override
+  public ShuffleBatchReader.Batch read(
+      @Nullable ShufflePosition startShufflePosition, @Nullable ShufflePosition endShufflePosition)
+      throws IOException {
+    @Nullable byte[] startPosition = ByteArrayShufflePosition.getPosition(startShufflePosition);
+    @Nullable byte[] endPosition = ByteArrayShufflePosition.getPosition(endShufflePosition);
+
+    ShuffleReader.ReadChunkResult result;
+    try (Closeable trackedReadState = tracker.enterState(readState)) {
+      result = reader.readIncludingPosition(startPosition, endPosition);
+    }
+    DataInputStream input = new DataInputStream(new ByteArrayInputStream(result.chunk));
+    ArrayList<ShuffleEntry> entries = new ArrayList<>();
+    while (input.available() > 0) {
+      entries.add(getShuffleEntry(input));
+    }
+    return new Batch(
+        entries,
+        result.nextStartPosition == null
+            ? null
+            : ByteArrayShufflePosition.of(result.nextStartPosition));
+  }
+
+  /**
+   * Extracts a ShuffleEntry by parsing bytes from a given InputStream.
+   *
+   * @param input stream to read from
+   * @return parsed ShuffleEntry
+   */
+  static ShuffleEntry getShuffleEntry(DataInputStream input) throws IOException {
+    byte[] position = getFixedLengthPrefixedByteArray(input);
+    byte[] key = getFixedLengthPrefixedByteArray(input);
+    byte[] skey = getFixedLengthPrefixedByteArray(input);
+    byte[] value = getFixedLengthPrefixedByteArray(input);
+    return new ShuffleEntry(ByteArrayShufflePosition.of(position), key, skey, value);
+  }
+
+  /**
+   * Extracts a length-prefix-encoded byte array from a given InputStream.
+   *
+   * @param dataInputStream stream to read from
+   * @return parsed byte array
+   */
+  static byte[] getFixedLengthPrefixedByteArray(DataInputStream dataInputStream)
+      throws IOException {
+    int length = dataInputStream.readInt();
+    if (length < 0) {
+      throw new IOException("invalid length: " + length);
+    }
+    byte[] data = new byte[length];
+    ByteStreams.readFully(dataInputStream, data);
+    return data;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombinePhase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombinePhase.java
new file mode 100644
index 0000000..5e81515
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombinePhase.java
@@ -0,0 +1,34 @@
+/*
+ * 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.dataflow.worker;
+
+/**
+ * The optimizer may split run the user combiner in 3 separate phases (ADD, MERGE, and EXTRACT), on
+ * separate VMs, as it sees fit. The CombinerPhase dictates which DoFn is actually running in the
+ * worker.
+ */
+// TODO: These strings are part of the service definition, and
+// should be added into the definition of the ParDoInstruction,
+// but the protiary definitions don't allow for enums yet.
+public class CombinePhase {
+  public static final String ALL = "all";
+  public static final String ADD = "add";
+  public static final String MERGE = "merge";
+  public static final String EXTRACT = "extract";
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java
new file mode 100644
index 0000000..12fa418
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java
@@ -0,0 +1,338 @@
+/*
+ * 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.dataflow.worker;
+
+import static org.apache.beam.runners.dataflow.util.Structs.getBytes;
+import static org.apache.beam.runners.dataflow.util.Structs.getString;
+
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.GlobalCombineFnRunner;
+import org.apache.beam.runners.core.GlobalCombineFnRunners;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.PropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+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.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.DoFnInfo;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * A {@link ParDoFnFactory} to create instances of user {@link CombineFn} according to
+ * specifications from the Dataflow service.
+ */
+class CombineValuesFnFactory implements ParDoFnFactory {
+
+  @Override
+  public ParDoFn create(
+      PipelineOptions options,
+      CloudObject cloudUserFn,
+      @Nullable List<SideInputInfo> sideInputInfos,
+      TupleTag<?> mainOutputTag,
+      Map<TupleTag<?>, Integer> outputTupleTagsToReceiverIndices,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+
+    Preconditions.checkArgument(
+        outputTupleTagsToReceiverIndices.size() == 1,
+        "expected exactly one output for CombineValuesFn");
+
+    Object deserializedFn =
+        SerializableUtils.deserializeFromByteArray(
+            getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized user fn");
+    Preconditions.checkArgument(deserializedFn instanceof AppliedCombineFn);
+    AppliedCombineFn<?, ?, ?, ?> combineFn = (AppliedCombineFn<?, ?, ?, ?>) deserializedFn;
+    Iterable<PCollectionView<?>> sideInputViews = combineFn.getSideInputViews();
+    SideInputReader sideInputReader =
+        executionContext.getSideInputReader(sideInputInfos, sideInputViews, operationContext);
+
+    // Get the combine phase, default to ALL. (The implementation
+    // doesn't have to split the combiner).
+    String phase = getString(cloudUserFn, WorkerPropertyNames.PHASE, CombinePhase.ALL);
+
+    DoFnInfo<?, ?> doFnInfo = getDoFnInfo(combineFn, sideInputReader, phase);
+    return new SimpleParDoFn(
+        options,
+        DoFnInstanceManagers.singleInstance(doFnInfo),
+        sideInputReader,
+        mainOutputTag,
+        outputTupleTagsToReceiverIndices,
+        executionContext.getStepContext(operationContext),
+        operationContext,
+        SimpleDoFnRunnerFactory.INSTANCE);
+  }
+
+  private static <K, InputT, AccumT, OutputT> DoFnInfo<?, ?> getDoFnInfo(
+      AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn,
+      SideInputReader sideInputReader,
+      String phase) {
+    switch (phase) {
+      case CombinePhase.ALL:
+        return CombineValuesDoFn.createDoFnInfo(combineFn, sideInputReader);
+      case CombinePhase.ADD:
+        return AddInputsDoFn.createDoFnInfo(combineFn, sideInputReader);
+      case CombinePhase.MERGE:
+        return MergeAccumulatorsDoFn.createDoFnInfo(combineFn, sideInputReader);
+      case CombinePhase.EXTRACT:
+        return ExtractOutputDoFn.createDoFnInfo(combineFn, sideInputReader);
+      default:
+        throw new IllegalArgumentException("phase must be one of 'all', 'add', 'merge', 'extract'");
+    }
+  }
+
+  /**
+   * The ALL phase is the unsplit combiner, in case combiner lifting is disabled or the optimizer
+   * chose not to lift this combiner.
+   */
+  private static class CombineValuesDoFn<K, InputT, OutputT>
+      extends DoFn<KV<K, Iterable<InputT>>, KV<K, OutputT>> {
+
+    private static <K, InputT, AccumT, OutputT> DoFnInfo<?, ?> createDoFnInfo(
+        AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn, SideInputReader sideInputReader) {
+      GlobalCombineFnRunner<InputT, AccumT, OutputT> combineFnRunner =
+          GlobalCombineFnRunners.create(combineFn.getFn());
+      DoFn<KV<K, Iterable<InputT>>, KV<K, OutputT>> doFn =
+          new CombineValuesDoFn<>(combineFnRunner, sideInputReader);
+
+      Coder<KV<K, Iterable<InputT>>> inputCoder = null;
+      if (combineFn.getKvCoder() != null) {
+        inputCoder =
+            KvCoder.of(
+                combineFn.getKvCoder().getKeyCoder(),
+                IterableCoder.of(combineFn.getKvCoder().getValueCoder()));
+      }
+      return DoFnInfo.forFn(
+          doFn,
+          combineFn.getWindowingStrategy(),
+          combineFn.getSideInputViews(),
+          inputCoder,
+          Collections.emptyMap(), // Not needed here.
+          new TupleTag<>(PropertyNames.OUTPUT));
+    }
+
+    private final GlobalCombineFnRunner<InputT, ?, OutputT> combineFnRunner;
+    private final SideInputReader sideInputReader;
+
+    private CombineValuesDoFn(
+        GlobalCombineFnRunner<InputT, ?, OutputT> combineFnRunner,
+        SideInputReader sideInputReader) {
+      this.combineFnRunner = combineFnRunner;
+      this.sideInputReader = sideInputReader;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      KV<K, Iterable<InputT>> kv = c.element();
+      c.output(
+          KV.of(
+              kv.getKey(),
+              applyCombineFn(combineFnRunner, kv.getValue(), window, c.getPipelineOptions())));
+    }
+
+    private <AccumT> OutputT applyCombineFn(
+        GlobalCombineFnRunner<InputT, AccumT, OutputT> combineFnRunner,
+        Iterable<InputT> inputs,
+        BoundedWindow window,
+        PipelineOptions options) {
+      List<BoundedWindow> windows = Collections.singletonList(window);
+      AccumT accum = combineFnRunner.createAccumulator(options, sideInputReader, windows);
+      for (InputT input : inputs) {
+        accum = combineFnRunner.addInput(accum, input, options, sideInputReader, windows);
+      }
+      return combineFnRunner.extractOutput(accum, options, sideInputReader, windows);
+    }
+  }
+
+  /*
+   * ADD phase: KV<K, Iterable<InputT>> -> KV<K, AccumT>.
+   */
+  private static class AddInputsDoFn<K, InputT, AccumT>
+      extends DoFn<KV<K, Iterable<InputT>>, KV<K, AccumT>> {
+
+    private final SideInputReader sideInputReader;
+
+    private static <K, InputT, AccumT, OutputT> DoFnInfo<?, ?> createDoFnInfo(
+        AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn, SideInputReader sideInputReader) {
+      GlobalCombineFnRunner<InputT, AccumT, OutputT> combineFnRunner =
+          GlobalCombineFnRunners.create(combineFn.getFn());
+      DoFn<KV<K, Iterable<InputT>>, KV<K, AccumT>> doFn =
+          new AddInputsDoFn<>(combineFnRunner, sideInputReader);
+
+      Coder<KV<K, Iterable<InputT>>> inputCoder = null;
+      if (combineFn.getKvCoder() != null) {
+        inputCoder =
+            KvCoder.of(
+                combineFn.getKvCoder().getKeyCoder(),
+                IterableCoder.of(combineFn.getKvCoder().getValueCoder()));
+      }
+      return DoFnInfo.forFn(
+          doFn,
+          combineFn.getWindowingStrategy(),
+          combineFn.getSideInputViews(),
+          inputCoder,
+          Collections.emptyMap(), // Not needed here.
+          new TupleTag<>(PropertyNames.OUTPUT));
+    }
+
+    private final GlobalCombineFnRunner<InputT, AccumT, ?> combineFnRunner;
+
+    private AddInputsDoFn(
+        GlobalCombineFnRunner<InputT, AccumT, ?> combineFnRunner, SideInputReader sideInputReader) {
+      this.combineFnRunner = combineFnRunner;
+      this.sideInputReader = sideInputReader;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      KV<K, Iterable<InputT>> kv = c.element();
+      K key = kv.getKey();
+
+      List<BoundedWindow> windows = Collections.singletonList(window);
+      AccumT accum =
+          combineFnRunner.createAccumulator(c.getPipelineOptions(), sideInputReader, windows);
+
+      for (InputT input : kv.getValue()) {
+        accum =
+            combineFnRunner.addInput(
+                accum, input, c.getPipelineOptions(), sideInputReader, windows);
+      }
+
+      c.output(KV.of(key, accum));
+    }
+  }
+
+  /*
+   * MERGE phase: KV<K, Iterable<AccumT>> -> KV<K, AccumT>.
+   */
+  private static class MergeAccumulatorsDoFn<K, AccumT>
+      extends DoFn<KV<K, Iterable<AccumT>>, KV<K, AccumT>> {
+
+    private final SideInputReader sideInputReader;
+
+    private static <K, InputT, AccumT, OutputT> DoFnInfo<?, ?> createDoFnInfo(
+        AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn, SideInputReader sideInputReader) {
+      GlobalCombineFnRunner<InputT, AccumT, OutputT> combineFnRunner =
+          GlobalCombineFnRunners.create(combineFn.getFn());
+      DoFn<KV<K, Iterable<AccumT>>, KV<K, AccumT>> doFn =
+          new MergeAccumulatorsDoFn<>(combineFnRunner, sideInputReader);
+
+      KvCoder<K, Iterable<AccumT>> inputCoder = null;
+      if (combineFn.getKvCoder() != null) {
+        inputCoder =
+            KvCoder.of(
+                combineFn.getKvCoder().getKeyCoder(),
+                IterableCoder.of(combineFn.getAccumulatorCoder()));
+      }
+      return DoFnInfo.forFn(
+          doFn,
+          combineFn.getWindowingStrategy(),
+          combineFn.getSideInputViews(),
+          inputCoder,
+          Collections.emptyMap(), // Not needed here.
+          new TupleTag<>(PropertyNames.OUTPUT));
+    }
+
+    private final GlobalCombineFnRunner<?, AccumT, ?> combineFnRunner;
+
+    private MergeAccumulatorsDoFn(
+        GlobalCombineFnRunner<?, AccumT, ?> combineFnRunner, SideInputReader sideInputReader) {
+      this.combineFnRunner = combineFnRunner;
+      this.sideInputReader = sideInputReader;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      KV<K, Iterable<AccumT>> kv = c.element();
+      K key = kv.getKey();
+      AccumT accum =
+          this.combineFnRunner.mergeAccumulators(
+              kv.getValue(),
+              c.getPipelineOptions(),
+              sideInputReader,
+              Collections.singletonList(window));
+      c.output(KV.of(key, accum));
+    }
+  }
+
+  /*
+   * EXTRACT phase: KV<K, AccumT> -> KV<K, OutputT>.
+   */
+  private static class ExtractOutputDoFn<K, AccumT, OutputT>
+      extends DoFn<KV<K, AccumT>, KV<K, OutputT>> {
+    private static <K, InputT, AccumT, OutputT> DoFnInfo<?, ?> createDoFnInfo(
+        AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn, SideInputReader sideInputReader) {
+      GlobalCombineFnRunner<InputT, AccumT, OutputT> combineFnRunner =
+          GlobalCombineFnRunners.create(combineFn.getFn());
+      DoFn<KV<K, AccumT>, KV<K, OutputT>> doFn =
+          new ExtractOutputDoFn<>(combineFnRunner, sideInputReader);
+
+      KvCoder<K, AccumT> inputCoder = null;
+      if (combineFn.getKvCoder() != null) {
+        inputCoder =
+            KvCoder.of(combineFn.getKvCoder().getKeyCoder(), combineFn.getAccumulatorCoder());
+      }
+      return DoFnInfo.forFn(
+          doFn,
+          combineFn.getWindowingStrategy(),
+          combineFn.getSideInputViews(),
+          inputCoder,
+          Collections.emptyMap(), // Not needed here.
+          new TupleTag<>(PropertyNames.OUTPUT));
+    }
+
+    private final GlobalCombineFnRunner<?, AccumT, OutputT> combineFnRunner;
+    private final SideInputReader sideInputReader;
+
+    private ExtractOutputDoFn(
+        GlobalCombineFnRunner<?, AccumT, OutputT> combineFnRunner,
+        SideInputReader sideInputReader) {
+      this.combineFnRunner = combineFnRunner;
+      this.sideInputReader = sideInputReader;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      KV<K, AccumT> kv = c.element();
+      K key = kv.getKey();
+      OutputT output =
+          this.combineFnRunner.extractOutput(
+              kv.getValue(),
+              c.getPipelineOptions(),
+              sideInputReader,
+              Collections.singletonList(window));
+      c.output(KV.of(key, output));
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java
new file mode 100644
index 0000000..f20bcc1
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java
@@ -0,0 +1,272 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.ApproximateReportedProgress;
+import com.google.api.services.dataflow.model.ApproximateSplitRequest;
+import com.google.api.services.dataflow.model.ConcatPosition;
+import com.google.api.services.dataflow.model.Source;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.CloudObjects;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.range.OffsetRangeTracker;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link NativeReader} that reads elements from a given set of encoded {@link Source}s. Creates
+ * {@link NativeReader}s for sources lazily, i.e. only when elements from the particular {@code
+ * NativeReader} are about to be read.
+ *
+ * <p>This class does does not cache {@link NativeReader}s and instead creates new set of {@link
+ * NativeReader}s for every new {@link ConcatIterator}. Because of this, multiple {@link
+ * ConcatIterator}s created using the same {@link ConcatReader} will not be able to share any state
+ * between each other. This design was chosen since keeping a large number of {@link NativeReader}
+ * objects alive within a single {@link ConcatReader} could be highly memory consuming.
+ *
+ * <p>For progress reporting and dynamic work rebalancing purposes, {@link ConcatIterator} uses a
+ * position of type {@link ConcatPosition}. Progress reporting and dynamic work rebalancing
+ * currently work only at the granularity of full sources being concatenated.
+ *
+ * @param <T> Type of the elements read by the {@link NativeReader}s.
+ */
+public class ConcatReader<T> extends NativeReader<T> {
+  private static final Logger LOG = LoggerFactory.getLogger(ConcatReader.class);
+
+  public static final String SOURCE_NAME = "ConcatSource";
+
+  private final List<Source> sources;
+  private final PipelineOptions options;
+  private final DataflowExecutionContext executionContext;
+  private final ReaderFactory readerFactory;
+  private final DataflowOperationContext operationContext;
+
+  /** Create a {@link ConcatReader} using a given list of encoded {@link Source}s. */
+  public ConcatReader(
+      ReaderFactory readerFactory,
+      PipelineOptions options,
+      DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext,
+      List<Source> sources) {
+    this.operationContext = operationContext;
+    Preconditions.checkNotNull(sources);
+    this.readerFactory = readerFactory;
+    this.sources = sources;
+    this.options = options;
+    this.executionContext = executionContext;
+  }
+
+  public Iterator<Source> getSources() {
+    return sources.iterator();
+  }
+
+  @Override
+  public ConcatIterator<T> iterator() throws IOException {
+    return new ConcatIterator<T>(
+        readerFactory, options, executionContext, operationContext, sources);
+  }
+
+  @VisibleForTesting
+  static class ConcatIterator<T> extends NativeReaderIterator<T> {
+    private int currentIteratorIndex = -1;
+    @Nullable private NativeReaderIterator<T> currentIterator = null;
+    private final List<Source> sources;
+    private final PipelineOptions options;
+    private final DataflowExecutionContext executionContext;
+    private final DataflowOperationContext operationContext;
+    private final OffsetRangeTracker rangeTracker;
+    private final ReaderFactory readerFactory;
+
+    public ConcatIterator(
+        ReaderFactory readerFactory,
+        PipelineOptions options,
+        DataflowExecutionContext executionContext,
+        DataflowOperationContext operationContext,
+        List<Source> sources) {
+      this.readerFactory = readerFactory;
+      this.sources = sources;
+      this.options = options;
+      this.executionContext = executionContext;
+      this.operationContext = operationContext;
+      this.rangeTracker = new OffsetRangeTracker(0, sources.size());
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      return advance();
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      while (true) {
+        // Invariant: we call currentIterator.start() immediately when opening an iterator
+        // (below). So if currentIterator != null, then start() has already been called on it.
+        if (currentIterator != null && currentIterator.advance()) {
+          // Happy case: current iterator has a next record.
+          return true;
+        }
+        // Now current iterator is either non-existent or exhausted.
+        // Close it, and try opening a new one.
+        if (currentIterator != null) {
+          currentIterator.close();
+          currentIterator = null;
+        }
+
+        if (!rangeTracker.tryReturnRecordAt(true, currentIteratorIndex + 1)) {
+          return false;
+        }
+        currentIteratorIndex++;
+        if (currentIteratorIndex == sources.size()) {
+          // All sources were read.
+          return false;
+        }
+
+        Source currentSource = sources.get(currentIteratorIndex);
+        try {
+          Coder<?> coder = null;
+          if (currentSource.getCodec() != null) {
+            coder =
+                CloudObjects.coderFromCloudObject(CloudObject.fromSpec(currentSource.getCodec()));
+          }
+          @SuppressWarnings("unchecked")
+          NativeReader<T> currentReader =
+              (NativeReader<T>)
+                  readerFactory.create(
+                      CloudObject.fromSpec(currentSource.getSpec()),
+                      coder,
+                      options,
+                      executionContext,
+                      operationContext);
+          currentIterator = currentReader.iterator();
+        } catch (Exception e) {
+          throw new IOException("Failed to create a reader for source: " + currentSource, e);
+        }
+        if (!currentIterator.start()) {
+          currentIterator.close();
+          currentIterator = null;
+          continue;
+        }
+        // Happy case: newly opened iterator has a first record.
+        return true;
+      }
+    }
+
+    @Override
+    public T getCurrent() throws NoSuchElementException {
+      if (currentIterator == null) {
+        throw new NoSuchElementException();
+      }
+      return currentIterator.getCurrent();
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (currentIterator != null) {
+        currentIterator.close();
+      }
+    }
+
+    @Override
+    public Progress getProgress() {
+      if (currentIteratorIndex < 0) {
+        // Reading has not been started yet.
+        return null;
+      }
+
+      if (currentIterator == null) {
+        // Reading has been finished.
+        throw new UnsupportedOperationException("Can't report progress of a finished iterator");
+      }
+
+      ConcatPosition concatPosition = new ConcatPosition();
+      concatPosition.setIndex(currentIteratorIndex);
+      Progress progressOfCurrentIterator = currentIterator.getProgress();
+      if (!(progressOfCurrentIterator instanceof SourceTranslationUtils.DataflowReaderProgress)) {
+        throw new IllegalArgumentException(
+            "Cannot process progress "
+                + progressOfCurrentIterator
+                + " since ConcatReader can only handle readers that generate a progress of type "
+                + "DataflowReaderProgress");
+      }
+      com.google.api.services.dataflow.model.Position positionOfCurrentIterator =
+          ((SourceTranslationUtils.DataflowReaderProgress) progressOfCurrentIterator)
+              .cloudProgress.getPosition();
+
+      if (positionOfCurrentIterator != null) {
+        concatPosition.setPosition(positionOfCurrentIterator);
+      }
+
+      ApproximateReportedProgress progress = new ApproximateReportedProgress();
+      com.google.api.services.dataflow.model.Position currentPosition =
+          new com.google.api.services.dataflow.model.Position();
+      currentPosition.setConcatPosition(concatPosition);
+      progress.setPosition(currentPosition);
+
+      return SourceTranslationUtils.cloudProgressToReaderProgress(progress);
+    }
+
+    @Override
+    public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) {
+      checkNotNull(splitRequest);
+
+      ApproximateSplitRequest splitProgress =
+          SourceTranslationUtils.splitRequestToApproximateSplitRequest(splitRequest);
+      com.google.api.services.dataflow.model.Position cloudPosition = splitProgress.getPosition();
+      if (cloudPosition == null) {
+        LOG.warn("Concat only supports split at a Position. Requested: {}", splitRequest);
+        return null;
+      }
+
+      ConcatPosition concatPosition = cloudPosition.getConcatPosition();
+      if (concatPosition == null) {
+        LOG.warn(
+            "ConcatReader only supports split at a ConcatPosition. Requested: {}", cloudPosition);
+        return null;
+      }
+
+      java.lang.Integer index = concatPosition.getIndex();
+      if (index == null) {
+        index = 0;
+      }
+
+      if (rangeTracker.trySplitAtPosition(index)) {
+        com.google.api.services.dataflow.model.Position positionToSplit =
+            new com.google.api.services.dataflow.model.Position();
+        positionToSplit.setConcatPosition(
+            new ConcatPosition().setIndex((int) rangeTracker.getStopPosition().longValue()));
+        return new DynamicSplitResultWithPosition(
+            SourceTranslationUtils.cloudPositionToReaderPosition(positionToSplit));
+      } else {
+        LOG.debug("Could not perform the dynamic split request " + splitRequest);
+        return null;
+      }
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java
new file mode 100644
index 0000000..c0f87d9
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java
@@ -0,0 +1,151 @@
+/*
+ * 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.dataflow.worker;
+
+import static org.apache.beam.runners.dataflow.util.Structs.getBoolean;
+import static org.apache.beam.runners.dataflow.util.Structs.getListOfMaps;
+import static org.apache.beam.runners.dataflow.util.Structs.getLong;
+import static org.apache.beam.runners.dataflow.util.Structs.getObject;
+
+import com.google.api.services.dataflow.model.Source;
+import com.google.api.services.dataflow.model.SourceMetadata;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.PropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Creates an {@link ConcatReader} from a {@link CloudObject} spec. */
+public class ConcatReaderFactory implements ReaderFactory {
+
+  private final ReaderRegistry registry;
+
+  private ConcatReaderFactory(ReaderRegistry registry) {
+    this.registry = registry;
+  }
+
+  /**
+   * Returns a new {@link ConcatReaderFactory} that will use the default {@link ReaderRegistry} to
+   * create sub-{@link NativeReader} instances.
+   */
+  public static ConcatReaderFactory withDefaultRegistry() {
+    return withRegistry(ReaderRegistry.defaultRegistry());
+  }
+
+  /**
+   * Returns a new {@link ConcatReaderFactory} that will use the provided {@link ReaderRegistry} to
+   * create sub-{@link NativeReader} instances.
+   */
+  public static ConcatReaderFactory withRegistry(ReaderRegistry registry) {
+    return new ConcatReaderFactory(registry);
+  }
+
+  @Override
+  public NativeReader<?> create(
+      CloudObject spec,
+      @Nullable Coder<?> coder,
+      @Nullable PipelineOptions options,
+      @Nullable DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    @SuppressWarnings("unchecked")
+    Coder<Object> typedCoder = (Coder<Object>) coder;
+    return createTyped(spec, typedCoder, options, executionContext, operationContext);
+  }
+
+  public <T> NativeReader<T> createTyped(
+      CloudObject spec,
+      @Nullable Coder<T> coder,
+      @Nullable PipelineOptions options,
+      @Nullable DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    List<Source> sources = getSubSources(spec);
+    return new ConcatReader<T>(registry, options, executionContext, operationContext, sources);
+  }
+
+  private static List<Source> getSubSources(CloudObject spec) throws Exception {
+    List<Source> subSources = new ArrayList<>();
+
+    // Get the list of sub-sources.
+    List<Map<String, Object>> subSourceDictionaries =
+        getListOfMaps(spec, WorkerPropertyNames.CONCAT_SOURCE_SOURCES, null);
+    if (subSourceDictionaries == null) {
+      return subSources;
+    }
+
+    for (Map<String, Object> subSourceDictionary : subSourceDictionaries) {
+      // Each sub-source is encoded as a dictionary that contains several properties.
+      subSources.add(createSourceFromDictionary(subSourceDictionary));
+    }
+
+    return subSources;
+  }
+
+  public static Source createSourceFromDictionary(Map<String, Object> dictionary) throws Exception {
+    Source source = new Source();
+
+    // Set spec
+    CloudObject subSourceSpec =
+        CloudObject.fromSpec(getObject(dictionary, PropertyNames.SOURCE_SPEC));
+    source.setSpec(subSourceSpec);
+
+    // Set encoding
+    CloudObject subSourceEncoding =
+        CloudObject.fromSpec(getObject(dictionary, PropertyNames.ENCODING, null));
+    if (subSourceEncoding != null) {
+      source.setCodec(subSourceEncoding);
+    }
+
+    // Set base specs
+    List<Map<String, Object>> subSourceBaseSpecs =
+        getListOfMaps(dictionary, WorkerPropertyNames.CONCAT_SOURCE_BASE_SPECS, null);
+    if (subSourceBaseSpecs != null) {
+      source.setBaseSpecs(subSourceBaseSpecs);
+    }
+
+    // Set metadata
+    SourceMetadata metadata = new SourceMetadata();
+    Boolean infinite = getBoolean(dictionary, PropertyNames.SOURCE_IS_INFINITE, null);
+    if (infinite != null) {
+      metadata.setInfinite(infinite);
+    }
+    Long estimatedSizeBytes = getLong(dictionary, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, null);
+    if (estimatedSizeBytes != null) {
+      metadata.setEstimatedSizeBytes(estimatedSizeBytes);
+    }
+    if (estimatedSizeBytes != null || infinite != null) {
+      source.setMetadata(metadata);
+    }
+
+    // Set doesNotNeedSplitting
+    Boolean doesNotNeedSplitting =
+        getBoolean(dictionary, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, null);
+    if (doesNotNeedSplitting != null) {
+      source.setDoesNotNeedSplitting(doesNotNeedSplitting);
+    }
+
+    return source;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ContextActivationObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ContextActivationObserver.java
new file mode 100644
index 0000000..e25ed95
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ContextActivationObserver.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow.worker;
+
+import com.google.auto.service.AutoService;
+import java.io.Closeable;
+import java.util.ServiceLoader;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker;
+import org.apache.beam.sdk.annotations.Experimental;
+
+/**
+ * A {@link ContextActivationObserver} provides a contract to register objects with the
+ * DataflowExecutionContext.
+ *
+ * <p>All methods of a {@link ContextActivationObserver} are required to be thread safe.
+ */
+public interface ContextActivationObserver extends Closeable {
+  /**
+   * Returns a {@link Closeable} based on the given {@link ExecutionStateTracker}. The closeable
+   * will be invoked when the {@link DataflowExecutionContext} is deactivated.
+   */
+  public Closeable activate(ExecutionStateTracker executionStateTracker);
+
+  /**
+   * {@link ContextActivationObserver}s can be automatically registered with this SDK by creating a
+   * {@link ServiceLoader} entry and a concrete implementation of this interface.
+   *
+   * <p>It is optional but recommended to use one of the many build time tools such as {@link
+   * AutoService} to generate the necessary META-INF files automatically.
+   */
+  @Experimental
+  public interface Registrar {
+    /**
+     * Returns a boolean indicating whether DataflowExecutionContext should register this observer
+     * or not.
+     */
+    boolean isEnabled();
+
+    /**
+     * Returns a {@link ContextActivationObserver} which will be registered by default within each
+     * {@link ContextActivationObserverRegistry registry} instance.
+     */
+    ContextActivationObserver getContextActivationObserver();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ContextActivationObserverRegistry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ContextActivationObserverRegistry.java
new file mode 100644
index 0000000..02ca492
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ContextActivationObserverRegistry.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow.worker;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ServiceLoader;
+import java.util.Set;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
+import org.apache.beam.sdk.util.common.ReflectHelpers.ObjectsClassComparator;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link ContextActivationObserverRegistry} allows creating a {@link ContextActivationObserver}.
+ */
+public class ContextActivationObserverRegistry {
+
+  static {
+    LoggerFactory.getLogger(ContextActivationObserverRegistry.class);
+  }
+
+  private static final List<ContextActivationObserver>
+      REGISTERED_CONTEXT_ACTIVATION_OBSERVER_FACTORIES;
+
+  static {
+    List<ContextActivationObserver> contextActivationObserversToRegister = new ArrayList<>();
+
+    // Enumerate all the ContextActivationObserver.Registrars, adding them to registry.
+    Set<ContextActivationObserver.Registrar> registrars =
+        Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
+    registrars.addAll(
+        Lists.newArrayList(
+            ServiceLoader.load(
+                ContextActivationObserver.Registrar.class, ReflectHelpers.findClassLoader())));
+
+    for (ContextActivationObserver.Registrar registrar : registrars) {
+      if (registrar.isEnabled()) {
+        contextActivationObserversToRegister.add(registrar.getContextActivationObserver());
+      }
+    }
+
+    REGISTERED_CONTEXT_ACTIVATION_OBSERVER_FACTORIES =
+        ImmutableList.copyOf(contextActivationObserversToRegister);
+  }
+
+  /**
+   * Creates a ContextActivationObserverRegistry containing registrations for all standard
+   * ContextActivationObservers part of the core Java Apache Beam SDK and also any registrations
+   * provided by {@link ContextActivationObserver.Registrar}s.
+   */
+  public static ContextActivationObserverRegistry createDefault() {
+    return new ContextActivationObserverRegistry();
+  }
+
+  /** The list of {@link ContextActivationObserver}s. */
+  private final List<ContextActivationObserver> contextActivationObservers;
+
+  private ContextActivationObserverRegistry() {
+    contextActivationObservers = new ArrayList<>(REGISTERED_CONTEXT_ACTIVATION_OBSERVER_FACTORIES);
+  }
+
+  /** Returns all {@link ContextActivationObserver}s registered with this registry. */
+  public List<ContextActivationObserver> getContextActivationObservers() {
+    return contextActivationObservers;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java
new file mode 100644
index 0000000..a84df1f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java
@@ -0,0 +1,155 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterStructuredName;
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.MetricShortId;
+import com.google.api.services.dataflow.model.ReportWorkItemStatusRequest;
+import com.google.api.services.dataflow.model.ReportWorkItemStatusResponse;
+import com.google.api.services.dataflow.model.WorkItemServiceState;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.common.base.MoreObjects;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.annotation.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mapping from counter names to short IDs.
+ *
+ * <p>This cache is non-evicting, and lives for the lifetime of the worker harness. This behavior is
+ * fine because the total number of unique counters is expected to be small and limited by the
+ * backend.
+ */
+public class CounterShortIdCache {
+  private static final Logger LOG = LoggerFactory.getLogger(CounterShortIdCache.class);
+  private Cache cache = new Cache();
+
+  /**
+   * Add any new short ids received to the table. The outgoing request will have the full counter
+   * updates, and the incoming responses have the associated short ids. By matching up short ids
+   * with the counters in order we can build a mapping of name -> short_id for future use.
+   */
+  public void storeNewShortIds(
+      final ReportWorkItemStatusRequest request, final ReportWorkItemStatusResponse reply) {
+    checkArgument(
+        request.getWorkItemStatuses() != null
+            && reply.getWorkItemServiceStates() != null
+            && request.getWorkItemStatuses().size() == reply.getWorkItemServiceStates().size(),
+        "RequestWorkItemStatus request and response are unbalanced, status: %s, states: %s",
+        request.getWorkItemStatuses(),
+        reply.getWorkItemServiceStates());
+
+    for (int i = 0; i < request.getWorkItemStatuses().size(); i++) {
+      WorkItemServiceState state = reply.getWorkItemServiceStates().get(i);
+      WorkItemStatus status = request.getWorkItemStatuses().get(i);
+      if (state.getMetricShortId() == null) {
+        continue;
+      }
+      checkArgument(
+          status.getCounterUpdates() != null,
+          "Response has shortids but no corresponding CounterUpdate");
+      for (MetricShortId shortIdMsg : state.getMetricShortId()) {
+        int metricIndex = MoreObjects.firstNonNull(shortIdMsg.getMetricIndex(), 0);
+        checkArgument(
+            metricIndex < status.getCounterUpdates().size(),
+            "Received aggregate index outside range of sent update %s >= %s",
+            shortIdMsg.getMetricIndex(),
+            status.getCounterUpdates().size());
+
+        CounterUpdate update = status.getCounterUpdates().get(metricIndex);
+        cache.insert(update, checkNotNull(shortIdMsg.getShortId(), "Shortid should be non-null"));
+      }
+    }
+  }
+
+  /**
+   * If any aggregates match a short id in the table, replace their name and type with the short id.
+   */
+  public void shortenIdsIfAvailable(@Nullable java.util.List<CounterUpdate> counters) {
+    if (counters == null) {
+      return;
+    }
+    for (CounterUpdate update : counters) {
+      cache.shortenIdsIfAvailable(update);
+    }
+  }
+
+  private static class Cache<K> {
+    private final ConcurrentHashMap<String, Long> unstructuredShortIdMap =
+        new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<CounterStructuredName, Long> structuredShortIdMap =
+        new ConcurrentHashMap<>();
+
+    public void shortenIdsIfAvailable(CounterUpdate update) {
+      Long shortId;
+      if (update.getNameAndKind() != null) {
+        String name =
+            checkNotNull(
+                update.getNameAndKind().getName(), "Counter update name should be non-null");
+        shortId = unstructuredShortIdMap.get(name);
+      } else if (update.getStructuredNameAndMetadata() != null) {
+        CounterStructuredName name =
+            checkNotNull(
+                update.getStructuredNameAndMetadata().getName(),
+                "Counter update structured-name should be non-null");
+        shortId = structuredShortIdMap.get(name);
+      } else {
+        throw new IllegalArgumentException(
+            "CounterUpdate should have nameAndKind or structuredNameAndmetadata");
+      }
+
+      if (shortId != null) {
+        update.setNameAndKind(null);
+        update.setStructuredNameAndMetadata(null);
+        update.setShortId(shortId);
+      }
+    }
+
+    public void insert(CounterUpdate update, long shortId) {
+      Long oldValue;
+      if (update.getNameAndKind() != null) {
+        String name =
+            checkNotNull(
+                update.getNameAndKind().getName(), "Counter update name should be non-null");
+        oldValue = unstructuredShortIdMap.putIfAbsent(name, shortId);
+      } else if (update.getStructuredNameAndMetadata() != null) {
+        CounterStructuredName name =
+            checkNotNull(
+                update.getStructuredNameAndMetadata().getName(),
+                "Counter update structured-name should be non-null");
+        oldValue = structuredShortIdMap.putIfAbsent(name, shortId);
+      } else {
+        throw new IllegalArgumentException(
+            "CounterUpdate should have nameAndKind or structuredNameAndmetadata");
+      }
+
+      checkArgument(
+          oldValue == null || oldValue.equals(shortId),
+          "Received counter %s with incompatible short IDs. %s first ID and then %s",
+          update,
+          oldValue,
+          shortId);
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java
new file mode 100644
index 0000000..6199095
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java
@@ -0,0 +1,120 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.CloudObjects;
+import org.apache.beam.runners.dataflow.util.PropertyNames;
+import org.apache.beam.runners.dataflow.util.Structs;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * A {@link ParDoFnFactory} that creates a system {@link ParDoFn} responsible for limiting the users
+ * key space to a fixed key space used to limit the number of ISM shards. This is done by
+ * transforming {@code WindowedValue<KV<K, V>>} into {@code WindowedValue<KV<VarInt32, KV<KV<K,
+ * Window>, V>>>} by applying a hashing function on the user key {@code K} and building a sort key
+ * based upon the user key {@code K} and the window.
+ *
+ * <p>This {@link ParDoFnFactory} is part of an expansion of steps required to materialize ISM
+ * files. See <a href="go/dataflow-side-inputs">go/dataflow-side-inputs</a> for further details.
+ */
+public class CreateIsmShardKeyAndSortKeyDoFnFactory implements ParDoFnFactory {
+
+  @Override
+  public ParDoFn create(
+      PipelineOptions options,
+      CloudObject cloudUserFn,
+      List<SideInputInfo> sideInputInfos,
+      TupleTag<?> mainOutputTag,
+      Map<TupleTag<?>, Integer> outputTupleTagsToReceiverIndices,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    Coder<?> coder =
+        CloudObjects.coderFromCloudObject(
+            CloudObject.fromSpec(Structs.getObject(cloudUserFn, PropertyNames.ENCODING)));
+    checkState(
+        coder instanceof IsmRecordCoder,
+        "Expected to received an instanceof an IsmRecordCoder but got %s",
+        coder);
+    return new CreateIsmShardKeyAndSortKeyParDoFn((IsmRecordCoder<?>) coder);
+  }
+
+  private static class CreateIsmShardKeyAndSortKeyParDoFn<K, V> implements ParDoFn {
+
+    private final IsmRecordCoder<?> coder;
+    private Receiver receiver;
+
+    CreateIsmShardKeyAndSortKeyParDoFn(IsmRecordCoder<?> coder) {
+      this.coder = coder;
+    }
+
+    @Override
+    public void startBundle(Receiver... receivers) throws Exception {
+      checkState(
+          receivers.length == 1,
+          "%s.startBundle() called with %s receivers, expected exactly 1. "
+              + "This is a bug in the Dataflow service",
+          getClass().getSimpleName(),
+          receivers.length);
+      this.receiver = receivers[0];
+    }
+
+    @Override
+    public void processElement(Object untypedElem) throws Exception {
+      @SuppressWarnings("unchecked")
+      WindowedValue<KV<K, V>> elem = (WindowedValue) untypedElem;
+
+      K userKey = elem.getValue().getKey();
+      V userValue = elem.getValue().getValue();
+      int hashKey = coder.hash(ImmutableList.of(elem.getValue().getKey()));
+
+      // Explode all the windows the users values are in.
+      for (BoundedWindow window : elem.getWindows()) {
+        KV<K, BoundedWindow> sortKey = KV.of(userKey, window);
+        KV<KV<K, BoundedWindow>, V> valueWithSortKey = KV.of(sortKey, userValue);
+        // Note that the shuffle writer expects a KV<PrimaryKey, KV<SortKey, Value>> when sorting.
+        receiver.process(elem.withValue(KV.of(hashKey, valueWithSortKey)));
+      }
+    }
+
+    @Override
+    public void processTimers() {}
+
+    @Override
+    public void finishBundle() {}
+
+    @Override
+    public void abort() {}
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowApiUtils.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowApiUtils.java
new file mode 100644
index 0000000..a721e0d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowApiUtils.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.dataflow.worker;
+
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.JsonGenerator;
+import com.google.common.io.ByteStreams;
+import com.google.common.io.CountingOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.util.Transport;
+
+/** A utility class for generic interactions with the Google Cloud Dataflow API. */
+public final class DataflowApiUtils {
+  /**
+   * Determines the serialized size (in bytes) of the {@link GenericJson} object that will be
+   * serialized and sent to the Google Cloud Dataflow service API.
+   *
+   * <p>Uses only constant memory.
+   */
+  public static long computeSerializedSizeBytes(GenericJson object) throws IOException {
+    JsonFactory factory = object.getFactory();
+    if (factory == null) {
+      factory = Transport.getJsonFactory();
+    }
+
+    CountingOutputStream stream = new CountingOutputStream(ByteStreams.nullOutputStream());
+    JsonGenerator generator = null;
+    try {
+      generator = factory.createJsonGenerator(stream, StandardCharsets.UTF_8);
+      generator.serialize(object);
+      generator.close(); // also closes the stream.
+    } finally {
+      if (generator != null) {
+        generator.close();
+      }
+    }
+    return stream.getCount();
+  }
+
+  // Prevent construction of utility class.
+  private DataflowApiUtils() {}
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java
new file mode 100644
index 0000000..8ccf868
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java
@@ -0,0 +1,174 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.util.FluentBackoff;
+import org.apache.beam.sdk.io.FileSystems;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the batch harness for executing Dataflow jobs where the worker and user/Java SDK code are
+ * running together in the same process.
+ */
+public class DataflowBatchWorkerHarness {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowBatchWorkerHarness.class);
+  private final DataflowWorkerHarnessOptions pipelineOptions;
+
+  private DataflowBatchWorkerHarness(DataflowWorkerHarnessOptions pipelineOptions) {
+    this.pipelineOptions = pipelineOptions;
+    checkArgument(
+        !pipelineOptions.isStreaming(),
+        "%s instantiated with pipeline options indicating a streaming job",
+        getClass().getSimpleName());
+  }
+
+  /** Creates the worker harness and then runs it. */
+  public static void main(String[] args) throws Exception {
+    DataflowWorkerHarnessHelper.initializeLogging(DataflowBatchWorkerHarness.class);
+    DataflowWorkerHarnessOptions pipelineOptions =
+        DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions(
+            DataflowBatchWorkerHarness.class);
+    DataflowBatchWorkerHarness batchHarness = new DataflowBatchWorkerHarness(pipelineOptions);
+    DataflowWorkerHarnessHelper.configureLogging(pipelineOptions);
+    batchHarness.run();
+  }
+
+  public static DataflowBatchWorkerHarness from(DataflowWorkerHarnessOptions pipelineOptions) {
+    return new DataflowBatchWorkerHarness(pipelineOptions);
+  }
+
+  /** Initializes the worker and starts the actual read loop (in {@link #processWork}). */
+  public void run() throws InterruptedException {
+    // Configure standard file systems.
+    FileSystems.setDefaultPipelineOptions(pipelineOptions);
+
+    DataflowWorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG);
+    BatchDataflowWorker worker =
+        BatchDataflowWorker.forBatchIntrinsicWorkerHarness(client, pipelineOptions);
+
+    worker.startStatusServer();
+    processWork(pipelineOptions, worker, Sleeper.DEFAULT);
+  }
+
+  // ExponentialBackOff parameters for the task retry strategy.
+  private static final long BACKOFF_INITIAL_INTERVAL_MILLIS = 5000; // 5 second
+  @VisibleForTesting static final long BACKOFF_MAX_INTERVAL_MILLIS = 5 * 60 * 1000; // 5 min.
+
+  /** Helper for initializing the BackOff used for retries. */
+  private static BackOff createBackOff() {
+    return FluentBackoff.DEFAULT
+        .withInitialBackoff(Duration.millis(BACKOFF_INITIAL_INTERVAL_MILLIS))
+        .withMaxBackoff(Duration.millis(BACKOFF_MAX_INTERVAL_MILLIS))
+        .backoff();
+  }
+
+  private static int chooseNumberOfThreads(DataflowWorkerHarnessOptions pipelineOptions) {
+    if (pipelineOptions.getNumberOfWorkerHarnessThreads() != 0) {
+      return pipelineOptions.getNumberOfWorkerHarnessThreads();
+    }
+    return Math.max(Runtime.getRuntime().availableProcessors(), 1);
+  }
+
+  /** A thread that repeatedly fetches and processes work units from the Dataflow service. */
+  private static class WorkerThread implements Callable<Boolean> {
+    // sleeper is used to sleep the appropriate amount of time
+    WorkerThread(final BatchDataflowWorker worker, final Sleeper sleeper) {
+      this.worker = worker;
+      this.sleeper = sleeper;
+      this.backOff = createBackOff();
+    }
+
+    @Override
+    public Boolean call() {
+      boolean success = true;
+      try {
+        do { // We loop getting and processing work.
+          success = doWork();
+          if (success) {
+            backOff.reset();
+          }
+          // Sleeping a while if there is a problem with the work, then go on with the next work.
+        } while (success || BackOffUtils.next(sleeper, backOff));
+      } catch (IOException e) { // Failure of BackOff.
+        LOG.error("Already tried several attempts at working on tasks. Aborting.", e);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.error("Interrupted during thread execution or sleep.", e);
+      } catch (Throwable t) {
+        LOG.error("Thread {} died.", Thread.currentThread().getId(), t);
+      }
+      return false;
+    }
+
+    private boolean doWork() {
+      try {
+        LOG.debug("Thread starting getAndPerformWork.");
+        boolean success = worker.getAndPerformWork();
+        LOG.debug("{} processing one WorkItem.", success ? "Finished" : "Failed");
+        return success;
+      } catch (IOException e) { // If there is a problem getting work.
+        LOG.debug("There was a problem getting work.", e);
+        return false;
+      } catch (Exception e) { // These exceptions are caused by bugs within the SDK
+        LOG.error("There was an unhandled error caused by the Dataflow SDK.", e);
+        return false;
+      }
+    }
+
+    private final BatchDataflowWorker worker;
+    private final Sleeper sleeper;
+    private final BackOff backOff;
+  }
+
+  @VisibleForTesting
+  static void processWork(
+      DataflowWorkerHarnessOptions pipelineOptions,
+      final BatchDataflowWorker worker,
+      Sleeper sleeper)
+      throws InterruptedException {
+    int numThreads = chooseNumberOfThreads(pipelineOptions);
+    ExecutorService executor = pipelineOptions.getExecutorService();
+    final List<Callable<Boolean>> tasks = new ArrayList<>();
+
+    LOG.debug("Starting {} worker threads", numThreads);
+    // We start the appropriate number of threads.
+    for (int i = 0; i < numThreads; ++i) {
+      tasks.add(new WorkerThread(worker, sleeper));
+    }
+
+    LOG.debug("Waiting for {} worker threads", numThreads);
+    // We wait forever unless there is a big problem.
+    executor.invokeAll(tasks);
+    LOG.error("All threads died.");
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java
new file mode 100644
index 0000000..a0f0d4b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java
@@ -0,0 +1,449 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+import java.time.Duration;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.worker.counters.Counter;
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementExecutionTracker;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateSampler;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dataflow-specific implementation of {@link ElementExecutionTracker}.
+ *
+ * <p>Each processed element through each step in the fused stage gets tracked as a {@code
+ * ElementExecution}. Processing time is sampled using {@link ExecutionStateSampler} and distributed
+ * amongst all elements which are executed since the last sample period.
+ *
+ * <p>Elements are processed in potentially many fragments of execution as we move up and down the
+ * stage graph via outputting. Each fragment of execution is counted equally for attributing sampled
+ * processing time.
+ *
+ * <p>When an element is finished processing it is held in the {@code doneJournal} collection until
+ * the next sampling round in order to calculate the final processing time. Eventually the total
+ * element processing time is reported to the counter and the state is cleaned up.
+ *
+ * <p>This is not implemented in {@link ElementExecutionTracker} itself because it uses
+ * Beam/Dataflow-specific classes, such as {@link PipelineOptions} which cannot be shared with
+ * Flume.
+ */
+public class DataflowElementExecutionTracker extends ElementExecutionTracker {
+  @VisibleForTesting
+  public static final String TIME_PER_ELEMENT_EXPERIMENT = "time_per_element_counter";
+
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowElementExecutionTracker.class);
+
+  public static ElementExecutionTracker create(
+      CounterFactory counterFactory, PipelineOptions options) {
+    // TODO: Remove once feature has launched.
+    if (!hasExperiment(options, TIME_PER_ELEMENT_EXPERIMENT)) {
+      return ElementExecutionTracker.NO_OP_INSTANCE;
+    }
+
+    // TODO: Remove log statement when functionality is enabled by default.
+    LOG.info("{} counter enabled.", ElementExecutionTracker.COUNTER_NAME);
+
+    return new DataflowElementExecutionTracker(counterFactory);
+  }
+
+  private static boolean hasExperiment(PipelineOptions options, String experiment) {
+    List<String> experiments = options.as(DataflowPipelineDebugOptions.class).getExperiments();
+    return experiments != null && experiments.contains(experiment);
+  }
+
+  /**
+   * Tracking object for the execution of a single input element in a step.
+   *
+   * <p>Each {@link ElementExecution} instance represents a distinct element, implicitly represented
+   * by the instance identity. As such, {@link ElementExecution} instances compare using reference
+   * equality rather than value equality.
+   */
+  private static class ElementExecution {
+    /** Marker execution to represent when there is no element currently being processed. */
+    static final ElementExecution IDLE = new ElementExecution();
+
+    /** Only empty for {@see IDLE}. */
+    final Optional<NameContext> step;
+
+    ElementExecution(NameContext step) {
+      this.step = Optional.of(step);
+    }
+
+    /** Only used for {@see IDLE}. */
+    private ElementExecution() {
+      step = Optional.empty();
+    }
+
+    @Override
+    public String toString() {
+      if (this == ElementExecution.IDLE) {
+        return "IDLE_EXECUTION";
+      }
+
+      return MoreObjects.toStringHelper(this)
+          .add("step", step)
+          .add("executionId", System.identityHashCode(this))
+          .toString();
+    }
+  }
+
+  private final ExecutionJournalWriter executionWriter;
+  private final ExecutionJournalReader executionReader;
+
+  private DataflowElementExecutionTracker(CounterFactory counterFactory) {
+    ReaderWriterState readerWriterState = new ReaderWriterState();
+    executionWriter = new ExecutionJournalWriter(readerWriterState);
+    executionReader = new ExecutionJournalReader(counterFactory, readerWriterState);
+  }
+
+  @Override
+  public void enter(NameContext step) {
+    executionWriter.startProcessing(step);
+  }
+
+  @Override
+  public void exit() {
+    executionWriter.doneProcessing();
+  }
+
+  @Override
+  public void takeSample(long millisSinceLastSample) {
+    executionReader.takeSample(Duration.ofMillis(millisSinceLastSample));
+  }
+
+  /** State shared between the {@link ExecutionJournalWriter} and {@link ExecutionJournalReader}. */
+  private static class ReaderWriterState {
+    /**
+     * Journal of fragments of execution per element to count for attributing processing time. Each
+     * time we transition up or down the stage fusion graph we add an execution fragment for the
+     * currently processing element with an incremented snapshot version. Each snapshot version must
+     * have a representative value in the {@code executionJournal}, or {@see IDLE_EXECUTION} to
+     * represent completion of processing.
+     */
+    private final Journal<ElementExecution> executionJournal;
+
+    /**
+     * Elements which have completed processing but are still pending final timing calculation. The
+     * elements are moved here when they exit the {@code executionStack} and are held here until the
+     * counter value can be reported in the next sampling round.
+     */
+    private final Journal<ElementExecution> doneJournal;
+
+    /**
+     * Monotonically increasing snapshot version number which tracks the latest execution state
+     * ready to be reported. Snapshot versions are associated with values in the {@code
+     * executionJournal} and {@code doneJournal}.
+     *
+     * <p>This value written by the single {@link ExecutionJournalWriter} thread, and read by the
+     * {@link ExecutionJournalReader} thread.
+     */
+    private volatile long latestSnapshot;
+
+    private ReaderWriterState() {
+      executionJournal = new Journal<>();
+      doneJournal = new Journal<>();
+      latestSnapshot = 0L;
+    }
+  }
+
+  /**
+   * Writes journal entries on element processing state changes.
+   *
+   * <p>Write operations are executed by a single thread inside the stage operation hot loop and
+   * must be highly performant. {@link ExecutionJournalWriter} methods are thread-safe only for
+   * state shared with the {@link ExecutionJournalReader} thread.
+   */
+  private static class ExecutionJournalWriter {
+    /**
+     * Execution stack of processing elements. Elements are pushed onto the stack when we enter the
+     * process() function and popped on process() return. This stack mirrors the actual Java runtime
+     * stack and contains the step + element context in order to attribute sampled execution time.
+     */
+    private final Deque<ElementExecution> executionStack;
+
+    private final ReaderWriterState sharedState;
+
+    public ExecutionJournalWriter(ReaderWriterState sharedState) {
+      this.sharedState = sharedState;
+      this.executionStack = new ArrayDeque<>();
+
+      addExecution(ElementExecution.IDLE);
+    }
+
+    /** Create and journal a new {@link ElementExecution} to track a processing element. */
+    public void startProcessing(NameContext step) {
+      ElementExecution execution = new ElementExecution(step);
+      addExecution(execution);
+    }
+
+    private void addExecution(ElementExecution execution) {
+      long nextSnapshot = sharedState.latestSnapshot + 1;
+      executionStack.addLast(execution);
+      sharedState.executionJournal.add(execution, nextSnapshot);
+      sharedState.latestSnapshot = nextSnapshot;
+    }
+
+    /**
+     * Indicates that the execution thread has exited the process method for an element.
+     *
+     * <p>When an element is finished processing, it is popped from the execution stack, but will be
+     * tracked in the {@code doneJournal} collection until the next sampling round in order to
+     * account timing for the final fragment of execution.
+     */
+    public void doneProcessing() {
+      // The execution stack in initialized with the IDLE execution which should never be removed.
+      checkState(executionStack.size() > 1, "No processing elements currently tracked.");
+
+      ElementExecution execution = executionStack.removeLast();
+      long nextSnapshot = sharedState.latestSnapshot + 1;
+      sharedState.doneJournal.add(execution, nextSnapshot);
+      ElementExecution nextElement = executionStack.getLast();
+      sharedState.executionJournal.add(nextElement, nextSnapshot);
+      sharedState.latestSnapshot = nextSnapshot;
+    }
+  }
+
+  /**
+   * Accounts sampled time to processed elements based on execution journal entries.
+   *
+   * <p>Read operations are executed by a single thread in charge of state sampling, and are
+   * generally called more infrequently than {@link ExecutionJournalWriter} operations. {@link
+   * ExecutionJournalReader} methods are thread-safe only for state shared with the {@link
+   * ExecutionJournalWriter} thread.
+   */
+  private static class ExecutionJournalReader {
+    /**
+     * Accumulated execution time per element. Once an element has finished processing and execution
+     * time has been attributed, the total execution time is reported via the counter and removed
+     * from the collection.
+     */
+    private final Map<ElementExecution, Duration> executionTimes;
+
+    private final CounterFactory counterFactory;
+    private final ReaderWriterState sharedState;
+
+    /** Cache of per-step distribution counters. */
+    private final Map<NameContext, Counter<Long, ?>> counterCache;
+
+    public ExecutionJournalReader(CounterFactory counterFactory, ReaderWriterState sharedState) {
+      this.sharedState = sharedState;
+      executionTimes = new HashMap<>();
+      this.counterFactory = counterFactory;
+      counterCache = new HashMap<>();
+    }
+
+    /**
+     * Account the specified processing time duration to elements which have processed since the
+     * last sampling round, and report counters for completed elements.
+     */
+    public void takeSample(Duration sampleTime) {
+      long snapshot = sharedState.latestSnapshot;
+      attributeProcessingTime(sampleTime, snapshot);
+
+      // TODO: If possible, this should report tentative counter values before they are
+      // finalized. Currently, root steps in a stage fusion will not have counters reported until an
+      // element is completely processed through the stage subgraph.
+      reportCounters(snapshot);
+      pruneJournals(snapshot);
+    }
+
+    /**
+     * Attribute processing time to elements from {@code executionJournal} up to the specified
+     * snapshot.
+     */
+    private void attributeProcessingTime(Duration duration, long snapshot) {
+      // TODO: This algorithm is used to compute "per-element-processing-time" counter
+      // values, but a slightly different algorithm is used for msec counters. Values for both
+      // counters should be derived from the same algorithm to avoid unexpected discrepancies.
+
+      // Calculate total execution counts
+      int totalExecutions = 0;
+      Map<ElementExecution, Integer> executionsPerElement = new HashMap<>();
+      for (ElementExecution execution : sharedState.executionJournal.readUntil(snapshot)) {
+        totalExecutions++;
+        if (execution != ElementExecution.IDLE) {
+          executionsPerElement.compute(execution, (unused, count) -> count == null ? 1 : count + 1);
+        }
+      }
+
+      // Attribute processing time
+      final int totalExecutionsFinal = totalExecutions;
+      for (Map.Entry<ElementExecution, Integer> executionCount : executionsPerElement.entrySet()) {
+        executionTimes.compute(
+            executionCount.getKey(),
+            (unused, total) -> {
+              int numExecutions = executionCount.getValue();
+              Duration attributedSampleTime =
+                  duration.dividedBy(totalExecutionsFinal).multipliedBy(numExecutions);
+              return total == null ? attributedSampleTime : total.plus(attributedSampleTime);
+            });
+      }
+    }
+
+    /** Report counter values for done elements up to the given snapshot. */
+    private void reportCounters(long snapshot) {
+      // Report counter values for completed elements
+      for (ElementExecution execution : sharedState.doneJournal.readUntil(snapshot)) {
+        checkState(
+            execution.step.isPresent(),
+            "Unexpected execution in doneJournal with empty step: %s",
+            execution);
+        Counter<Long, ?> counter = getCounter(execution.step.get());
+        counter.addValue(executionTimes.get(execution).toMillis());
+      }
+    }
+
+    /** Retrieve the per-element processing time counter for the specified step. */
+    private Counter<Long, ?> getCounter(NameContext step) {
+      return counterCache.computeIfAbsent(
+          step,
+          s ->
+              counterFactory.distribution(
+                  ElementExecutionTracker.COUNTER_NAME.withOriginalName(s)));
+    }
+
+    /**
+     * Prune journal entries which have been accounted.
+     *
+     * @param snapshot Snapshot value which has been accounted.
+     */
+    private void pruneJournals(long snapshot) {
+      sharedState.doneJournal.pruneUntil(snapshot);
+
+      // Keep the currently executing element in the journal so its remaining execution time
+      // is counted in the next sampling round.
+      sharedState.executionJournal.pruneUntil(snapshot - 1);
+    }
+  }
+
+  /**
+   * Concurrent queue-based data structure for passing journaled events between a single journal
+   * reader and single journal writer.
+   *
+   * <p>Each event is journaled with an externally-managed snapshot version. Snapshot versions are
+   * unique and monotonically increasing.
+   */
+  private static class Journal<T> {
+    private final ConcurrentLinkedQueue<SnapshottedItem<T>> queue;
+    private long maxSnapshot;
+
+    public Journal() {
+      queue = new ConcurrentLinkedQueue<>();
+      maxSnapshot = Long.MIN_VALUE;
+    }
+
+    /** Add a new event to the journal with the given snapshot. */
+    public void add(T item, long snapshot) {
+      if (snapshot <= maxSnapshot) {
+        throw new IllegalArgumentException(
+            String.format(
+                "Timestamps must be monotonically increasing. "
+                    + "Specified snapshot '%d' is not greater than max snapshot '%d'",
+                snapshot, maxSnapshot));
+      }
+      queue.add(new SnapshottedItem<>(item, snapshot));
+      maxSnapshot = snapshot;
+    }
+
+    /**
+     * Retrieve an iterable of events from the journal up to and including the specified snapshot
+     * version.
+     *
+     * <p>Returns an empty iterable if the journal is empty or there are no events less than the
+     * specified snapshot version.
+     *
+     * <p>The returned iterable returns a view of the queue at the time of construction, consistent
+     * with {@link ConcurrentLinkedQueue#iterator()}.
+     */
+    public Iterable<T> readUntil(long snapshot) {
+      // Implement our own iterator in order to short-circuit early once we've seen a snapshot
+      // version greater than the value specified.
+      // Streams will have takeWhile() in JDK9, and Guava may some day have
+      // Iterables.limit(Predicate<T>): https://github.com/google/guava/issues/477
+      PeekingIterator<SnapshottedItem<T>> iter = Iterators.peekingIterator(queue.iterator());
+      return () ->
+          new Iterator<T>() {
+            @Override
+            public boolean hasNext() {
+              return iter.hasNext() && iter.peek().snapshot <= snapshot;
+            }
+
+            @Override
+            public T next() {
+              SnapshottedItem<T> next = iter.next();
+              checkValidSnapshot(next);
+              return next.item;
+            }
+
+            @Override
+            public void remove() {
+              iter.remove();
+            }
+
+            private void checkValidSnapshot(SnapshottedItem<T> next) {
+              if (next.snapshot > snapshot) {
+                throw new NoSuchElementException();
+              }
+            }
+          };
+    }
+
+    /** Prune events from the journal up to and including the specified snapshot. */
+    public void pruneUntil(long snapshot) {
+      Iterator<SnapshottedItem<T>> iterator = queue.iterator();
+      while (iterator.hasNext()) {
+        if (iterator.next().snapshot <= snapshot) {
+          iterator.remove();
+        } else {
+          break;
+        }
+      }
+    }
+
+    private static class SnapshottedItem<E> {
+      private final E item;
+      private final long snapshot;
+
+      public SnapshottedItem(E item, long snapshot) {
+        this.item = item;
+        this.snapshot = snapshot;
+      }
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java
new file mode 100644
index 0000000..4ec1046
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java
@@ -0,0 +1,277 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.collect.Iterables;
+import com.google.common.io.Closer;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.NullSideInputReader;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.core.StepContext;
+import org.apache.beam.runners.core.TimerInternals.TimerData;
+import org.apache.beam.runners.dataflow.worker.DataflowExecutionContext.DataflowStepContext;
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateSampler;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.metrics.MetricsContainer;
+import org.apache.beam.sdk.metrics.MetricsEnvironment;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.joda.time.Instant;
+
+/** Execution context for the Dataflow worker. */
+public abstract class DataflowExecutionContext<T extends DataflowStepContext> {
+
+  private final CounterFactory counterFactory;
+  private final MetricsContainerRegistry<?> metricsContainerRegistry;
+  private final ExecutionStateTracker executionStateTracker;
+  protected final ExecutionStateRegistry executionStateRegistry;
+  // Desired limit on amount of data sinked. Cumulative
+  // across all the sinks, when there are more than one sinks.
+  private final long sinkByteLimit;
+  private long bytesSinked = 0;
+
+  public DataflowExecutionContext(
+      CounterFactory counterFactory,
+      MetricsContainerRegistry<?> metricsRegistry,
+      DataflowExecutionStateTracker executionStateTracker,
+      ExecutionStateRegistry executionStateRegistry,
+      long sinkByteLimit) {
+    this.counterFactory = counterFactory;
+    this.metricsContainerRegistry = metricsRegistry;
+    this.executionStateTracker = executionStateTracker;
+    this.executionStateRegistry = executionStateRegistry;
+    this.sinkByteLimit = sinkByteLimit;
+  }
+
+  // Created step contexts, keyed by step name
+  private Map<String, T> cachedStepContexts = new LinkedHashMap<>();
+
+  /**
+   * Returns a {@link SideInputReader} based on {@link SideInputInfo} descriptors and {@link
+   * PCollectionView PCollectionViews}.
+   *
+   * <p>If side input source metadata is provided by the service in {@link SideInputInfo
+   * sideInputInfos}, we request a {@link SideInputReader} from the {@code executionContext} using
+   * that info. If no side input source metadata is provided but the DoFn expects side inputs, as a
+   * fallback, we request a {@link SideInputReader} based only on the expected views.
+   *
+   * <p>These cases are not disjoint: Whenever a {@link GroupAlsoByWindowFn} takes side inputs,
+   * {@code doFnInfo.getSideInputViews()} should be non-empty.
+   *
+   * <p>A note on the behavior of the Dataflow service: Today, the first case corresponds to batch
+   * mode, while the fallback corresponds to streaming mode.
+   */
+  public SideInputReader getSideInputReader(
+      @Nullable Iterable<? extends SideInputInfo> sideInputInfos,
+      @Nullable Iterable<? extends PCollectionView<?>> views,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    if (sideInputInfos != null && !Iterables.isEmpty(sideInputInfos)) {
+      return getSideInputReader(sideInputInfos, operationContext);
+    } else if (views != null && !Iterables.isEmpty(views)) {
+      return getSideInputReaderForViews(views);
+    } else {
+      return NullSideInputReader.empty();
+    }
+  }
+
+  public CounterFactory getCounterFactory() {
+    return counterFactory;
+  }
+
+  /** Returns a collection view of all of the {@link StepContext}s. */
+  public Collection<? extends T> getAllStepContexts() {
+    return Collections.unmodifiableCollection(cachedStepContexts.values());
+  }
+
+  /**
+   * A hint for currently executing step if the context prefers stopping processing more input since
+   * the sinks are 'full'. This is polled by readers to stop consuming more records, when they can.
+   * Currently the hint is set only by the sinks in streaming.
+   */
+  boolean isSinkFullHintSet() {
+    return bytesSinked >= sinkByteLimit;
+    // In addition to hint from the sinks, we could consider other factors likes global memory
+    // pressure.
+    // TODO: We should have state bytes also to contribute to this hint, otherwise,
+    //               the state size might grow unbounded.
+  }
+
+  /**
+   * Sets a flag to indicate that a sink has enough data written to it. This hint is read by
+   * upstream producers to stop producing if they can. Mainly used in streaming.
+   */
+  void reportBytesSinked(long bytes) {
+    bytesSinked += bytes;
+  }
+
+  protected void clearSinkFullHint() {
+    // Cleared in Streaming when the context is reused for new work item.
+    bytesSinked = 0;
+  }
+
+  /**
+   * Returns a {@link SideInputReader} for all the side inputs described in the given {@link
+   * SideInputInfo} descriptors.
+   */
+  protected abstract SideInputReader getSideInputReader(
+      Iterable<? extends SideInputInfo> sideInputInfos, DataflowOperationContext operationContext)
+      throws Exception;
+
+  protected abstract T createStepContext(DataflowOperationContext operationContext);
+
+  // TODO: Move StepContext creation to the OperationContext.
+  public T getStepContext(DataflowOperationContext operationContext) {
+    NameContext nameContext = operationContext.nameContext();
+    T context = cachedStepContexts.get(nameContext.systemName());
+    if (context == null) {
+      context = createStepContext(operationContext);
+      cachedStepContexts.put(nameContext.systemName(), context);
+    }
+    return context;
+  }
+
+  /**
+   * Returns a {@link SideInputReader} for all the provided views, where the execution context
+   * itself knows how to read data for the view.
+   */
+  protected abstract SideInputReader getSideInputReaderForViews(
+      Iterable<? extends PCollectionView<?>> views) throws Exception;
+
+  /** Dataflow specific {@link StepContext}. */
+  public abstract static class DataflowStepContext implements StepContext {
+    private final NameContext nameContext;
+
+    public DataflowStepContext(NameContext nameContext) {
+      this.nameContext = nameContext;
+    }
+
+    public NameContext getNameContext() {
+      return nameContext;
+    }
+
+    /**
+     * Returns the next fired timer for this step.
+     *
+     * <p>The {@code windowCoder} is passed here as it is more convenient than doing so when the
+     * {@link DataflowStepContext} is created.
+     */
+    @Nullable
+    public abstract <W extends BoundedWindow> TimerData getNextFiredTimer(Coder<W> windowCoder);
+
+    public abstract <W extends BoundedWindow> void setStateCleanupTimer(
+        String timerId, W window, Coder<W> windowCoder, Instant cleanupTime);
+
+    public abstract DataflowStepContext namespacedToUser();
+  }
+
+  /**
+   * Creates the context for an operation with the stage this execution context belongs to.
+   *
+   * @param nameContext the set of names identifying the operation
+   */
+  public DataflowOperationContext createOperationContext(NameContext nameContext) {
+    MetricsContainer metricsContainer =
+        metricsContainerRegistry.getContainer(
+            checkNotNull(
+                nameContext.originalName(),
+                "All operations must have an original name, but %s doesn't.",
+                nameContext));
+    return new DataflowOperationContext(
+        counterFactory,
+        nameContext,
+        metricsContainer,
+        executionStateTracker,
+        executionStateRegistry);
+  }
+
+  protected MetricsContainerRegistry<?> getMetricsContainerRegistry() {
+    return metricsContainerRegistry;
+  }
+
+  protected ExecutionStateRegistry getExecutionStateRegistry() {
+    return executionStateRegistry;
+  }
+
+  public ExecutionStateTracker getExecutionStateTracker() {
+    return executionStateTracker;
+  }
+
+  /**
+   * An extension of {@link ExecutionStateTracker} that also installs a {@link MetricsContainer}
+   * using the current state to locate the {@link MetricsEnvironment}.
+   */
+  public static class DataflowExecutionStateTracker extends ExecutionStateTracker {
+
+    private final DataflowOperationContext.DataflowExecutionState otherState;
+    private final ContextActivationObserverRegistry contextActivationObserverRegistry;
+    private final String workItemId;
+
+    public DataflowExecutionStateTracker(
+        ExecutionStateSampler sampler,
+        DataflowOperationContext.DataflowExecutionState otherState,
+        CounterFactory counterFactory,
+        PipelineOptions options,
+        String workItemId) {
+      super(sampler, DataflowElementExecutionTracker.create(counterFactory, options));
+      this.otherState = otherState;
+      this.workItemId = workItemId;
+      this.contextActivationObserverRegistry = ContextActivationObserverRegistry.createDefault();
+    }
+
+    @Override
+    public Closeable activate() {
+      Closer closer = Closer.create();
+      try {
+        closer.register(super.activate());
+        for (ContextActivationObserver p :
+            contextActivationObserverRegistry.getContextActivationObservers()) {
+          closer.register(p.activate(this));
+        }
+        closer.register(enterState(otherState));
+        return closer;
+      } catch (Exception e) {
+        try {
+          closer.close();
+        } catch (IOException suppressed) {
+          // Shouldn't happen -- none of the things being closed actually throw.
+          e.addSuppressed(suppressed);
+        }
+        throw e;
+      }
+    }
+
+    public String getWorkItemId() {
+      return this.workItemId;
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMapTaskExecutor.java
new file mode 100644
index 0000000..086cc17
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMapTaskExecutor.java
@@ -0,0 +1,39 @@
+/*
+ * 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.dataflow.worker;
+
+import java.util.List;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.MapTaskExecutor;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Operation;
+
+/** A {@link DataflowWorkExecutor} that is also a {@link MapTaskExecutor}. */
+public abstract class DataflowMapTaskExecutor extends MapTaskExecutor
+    implements DataflowWorkExecutor {
+
+  /** @deprecated subclasses should move to composition instead of inheritance, make this private */
+  @Deprecated
+  protected DataflowMapTaskExecutor(
+      List<Operation> operations,
+      CounterSet counters,
+      ExecutionStateTracker executionStateTracker) {
+    super(operations, counters, executionStateTracker);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMapTaskExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMapTaskExecutorFactory.java
new file mode 100644
index 0000000..b93d41d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMapTaskExecutorFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.api.services.dataflow.model.MapTask;
+import com.google.common.graph.MutableNetwork;
+import java.util.function.Supplier;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import org.apache.beam.runners.dataflow.worker.graph.Edges.Edge;
+import org.apache.beam.runners.dataflow.worker.graph.Nodes.Node;
+import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
+import org.apache.beam.runners.fnexecution.data.FnDataService;
+import org.apache.beam.runners.fnexecution.state.StateDelegator;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Creates a {@link DataflowMapTaskExecutor} from a {@link MapTask} definition. */
+public interface DataflowMapTaskExecutorFactory {
+
+  /**
+   * Creates a new {@link DataflowMapTaskExecutor} from the given {@link MapTask} definition using
+   * the provided {@link ReaderFactory} as well as a wide variety of other contextual information.
+   */
+  DataflowMapTaskExecutor create(
+      InstructionRequestHandler instructionRequestHandler,
+      FnDataService beamFnDataService,
+      Endpoints.ApiServiceDescriptor dataApiServiceDescriptor,
+      StateDelegator beamFnStateDelegator,
+      MutableNetwork<Node, Edge> network,
+      PipelineOptions options,
+      String stageName,
+      ReaderFactory readerFactory,
+      SinkFactory sinkFactory,
+      DataflowExecutionContext<?> executionContext,
+      CounterSet counterSet,
+      Supplier<String> idGenerator);
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
new file mode 100644
index 0000000..7711ae3
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
@@ -0,0 +1,62 @@
+/*
+ * 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.dataflow.worker;
+
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.Gauge;
+import org.apache.beam.sdk.metrics.MetricName;
+import org.apache.beam.sdk.metrics.MetricsContainer;
+import org.apache.beam.sdk.metrics.MetricsEnvironment;
+
+/**
+ * An implementation of {@link MetricsContainer} that reads the current execution state (tracked in
+ * a field) to determine the current step. This allows the {@link MetricsEnvironment} to only be
+ * updated once on entry to the entire stage, rather than in between every step.
+ */
+public class DataflowMetricsContainer implements MetricsContainer {
+
+  private final ExecutionStateTracker executionStateTracker;
+
+  public DataflowMetricsContainer(ExecutionStateTracker executionStateTracker) {
+    this.executionStateTracker = executionStateTracker;
+  }
+
+  private MetricsContainer getCurrentContainer() {
+    DataflowOperationContext.DataflowExecutionState executionState =
+        (DataflowOperationContext.DataflowExecutionState) executionStateTracker.getCurrentState();
+    return executionState.getMetricsContainer();
+  }
+
+  @Override
+  public Counter getCounter(MetricName metricName) {
+    return getCurrentContainer().getCounter(metricName);
+  }
+
+  @Override
+  public Distribution getDistribution(MetricName metricName) {
+    return getCurrentContainer().getDistribution(metricName);
+  }
+
+  @Override
+  public Gauge getGauge(MetricName metricName) {
+    return getCurrentContainer().getGauge(metricName);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java
new file mode 100644
index 0000000..72304a0
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java
@@ -0,0 +1,300 @@
+/*
+ * 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.dataflow.worker;
+
+import static org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor.longToSplitInt;
+
+import com.google.api.services.dataflow.model.CounterMetadata;
+import com.google.api.services.dataflow.model.CounterStructuredName;
+import com.google.api.services.dataflow.model.CounterStructuredNameAndMetadata;
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import java.io.Closeable;
+import java.util.logging.Level;
+import java.util.logging.LogRecord;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.SimpleDoFnRunner;
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingInitializer;
+import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler;
+import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker.ExecutionState;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OperationContext;
+import org.apache.beam.sdk.metrics.MetricsContainer;
+import org.joda.time.Duration;
+import org.joda.time.format.PeriodFormatter;
+import org.joda.time.format.PeriodFormatterBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link OperationContext} that manages the current {@link ExecutionState} to ensure the
+ * start/process/finish/abort states are properly tracked.
+ */
+public class DataflowOperationContext implements OperationContext {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowOperationContext.class);
+
+  private final CounterFactory counterFactory;
+
+  private final NameContext nameContext;
+  private final ProfileScope profileScope;
+  private final ExecutionState startState;
+  private final ExecutionState processElementState;
+  private final ExecutionState processTimersState;
+  private final ExecutionState finishState;
+  private final ExecutionState abortState;
+  private final MetricsContainer metricsContainer;
+  private final ExecutionStateTracker executionStateTracker;
+  private final ExecutionStateRegistry executionStateRegistry;
+
+  DataflowOperationContext(
+      CounterFactory counterFactory,
+      NameContext nameContext,
+      MetricsContainer metricsContainer,
+      ExecutionStateTracker executionStateTracker,
+      ExecutionStateRegistry executionStateRegistry) {
+    this(
+        counterFactory,
+        nameContext,
+        metricsContainer,
+        executionStateTracker,
+        executionStateRegistry,
+        ScopedProfiler.INSTANCE);
+  }
+
+  @VisibleForTesting
+  DataflowOperationContext(
+      CounterFactory counterFactory,
+      NameContext nameContext,
+      MetricsContainer metricsContainer,
+      ExecutionStateTracker executionStateTracker,
+      ExecutionStateRegistry executionStateRegistry,
+      ScopedProfiler scopedProfiler) {
+    this.counterFactory = counterFactory;
+    this.nameContext = nameContext;
+    this.metricsContainer = metricsContainer;
+    this.executionStateTracker = executionStateTracker;
+    this.executionStateRegistry = executionStateRegistry;
+
+    profileScope = scopedProfiler.registerScope(nameContext.originalName());
+    startState = newExecutionState(ExecutionStateTracker.START_STATE_NAME);
+    processElementState = newExecutionState(ExecutionStateTracker.PROCESS_STATE_NAME);
+    processTimersState = newExecutionState(ExecutionStateTracker.PROCESS_TIMERS_STATE_NAME);
+    finishState = newExecutionState(ExecutionStateTracker.FINISH_STATE_NAME);
+    abortState = newExecutionState(ExecutionStateTracker.ABORT_STATE_NAME);
+  }
+
+  public ExecutionState newExecutionState(String stateName) {
+    return executionStateRegistry.getState(nameContext, stateName, metricsContainer, profileScope);
+  }
+
+  @Override
+  public Closeable enterStart() {
+    return enter(startState);
+  }
+
+  @Override
+  public Closeable enterProcess() {
+    return enter(processElementState);
+  }
+
+  @Override
+  public Closeable enterProcessTimers() {
+    // TODO: It could be useful to capture enough context to report per-timer
+    // msec counters.
+    return enter(processTimersState);
+  }
+
+  @Override
+  public Closeable enterFinish() {
+    return enter(finishState);
+  }
+
+  @Override
+  public Closeable enterAbort() {
+    return enter(abortState);
+  }
+
+  @Override
+  public CounterFactory counterFactory() {
+    return counterFactory;
+  }
+
+  @Override
+  public NameContext nameContext() {
+    return nameContext;
+  }
+
+  public MetricsContainer metricsContainer() {
+    return metricsContainer;
+  }
+
+  private Closeable enter(ExecutionState state) {
+    return executionStateTracker.enterState(state);
+  }
+
+  /**
+   * An {@link DataflowExecutionState} represents the current state of an execution thread. It also
+   * tracks the {@link MetricsContainer} and {@link ProfileScope} for this state, along with
+   * Dataflow-specific data and functionality.
+   */
+  public abstract static class DataflowExecutionState extends ExecutionState {
+
+    /**
+     * For states that represent consumption / output of IO, this represents the step running when
+     * the IO is triggered.
+     */
+    @Nullable private final String requestingStepName;
+
+    /**
+     * For states that represent consumption of IO, this represents the index of the PCollection
+     * that is associated to the IO performed (e.g. for side input reading, this is the index of the
+     * side input).
+     */
+    @Nullable private final Integer inputIndex;
+
+    private final ProfileScope profileScope;
+    @Nullable private final MetricsContainer metricsContainer;
+
+    public DataflowExecutionState(
+        NameContext nameContext,
+        String stateName,
+        @Nullable String requestingStepName,
+        @Nullable Integer inputIndex,
+        @Nullable MetricsContainer metricsContainer,
+        ProfileScope profileScope) {
+      super(nameContext, stateName);
+      this.requestingStepName = requestingStepName;
+      this.inputIndex = inputIndex;
+      this.profileScope = Preconditions.checkNotNull(profileScope);
+      this.metricsContainer = metricsContainer;
+    }
+
+    @Override
+    public void onActivate(boolean unusedPushing) {
+      profileScope.activate();
+    }
+
+    public ProfileScope getProfileScope() {
+      return profileScope;
+    }
+
+    private static final ImmutableSet<String> FRAMEWORK_CLASSES =
+        ImmutableSet.of(SimpleDoFnRunner.class.getName(), DoFnInstanceManagers.class.getName());
+
+    private String getLullMessage(Thread trackedThread, Duration millis) {
+      StringBuilder message = new StringBuilder();
+      message.append("Processing stuck");
+      if (getStepName() != null) {
+        message.append(" in step ").append(getStepName().userName());
+      }
+      message
+          .append(" for at least ")
+          .append(formatDuration(millis))
+          .append(" without outputting or completing in state ")
+          .append(getStateName());
+      message.append("\n");
+
+      StackTraceElement[] fullTrace = trackedThread.getStackTrace();
+      for (StackTraceElement e : fullTrace) {
+        if (FRAMEWORK_CLASSES.contains(e.getClassName())) {
+          break;
+        }
+        message.append("  at ").append(e).append("\n");
+      }
+      return message.toString();
+    }
+
+    @Override
+    public void reportLull(Thread trackedThread, long millis) {
+      // If we're not logging warnings, nothing to report.
+      if (!LOG.isWarnEnabled()) {
+        return;
+      }
+
+      // Since the lull reporting executes in the sampler thread, it won't automatically inherit the
+      // context of the current step. To ensure things are logged correctly, we get the currently
+      // registered DataflowWorkerLoggingHandler and log directly in the desired context.
+      LogRecord logRecord =
+          new LogRecord(Level.WARNING, getLullMessage(trackedThread, Duration.millis(millis)));
+      logRecord.setLoggerName(DataflowOperationContext.LOG.getName());
+
+      // Publish directly in the context of this specific ExecutionState.
+      DataflowWorkerLoggingInitializer.getLoggingHandler().publish(this, logRecord);
+    }
+
+    @Nullable
+    public MetricsContainer getMetricsContainer() {
+      return metricsContainer;
+    }
+
+    @Nullable
+    public abstract CounterUpdate extractUpdate(boolean isFinalUpdate);
+
+    protected CounterUpdate createUpdate(boolean isCumulative, long value) {
+      CounterStructuredName name =
+          new CounterStructuredName()
+              .setName(getStateName() + "-msecs")
+              .setOrigin("SYSTEM")
+              .setExecutionStepName(getStepName().stageName());
+
+      if (getStepName().originalName() != null) {
+        name.setOriginalStepName(getStepName().originalName());
+      }
+      if (requestingStepName != null) {
+        name.setOriginalRequestingStepName(requestingStepName);
+      }
+      if (inputIndex != null && inputIndex > 0) {
+        name.setInputIndex(inputIndex);
+      }
+
+      return new CounterUpdate()
+          .setStructuredNameAndMetadata(
+              new CounterStructuredNameAndMetadata()
+                  .setName(name)
+                  .setMetadata(new CounterMetadata().setKind("SUM")))
+          .setCumulative(isCumulative)
+          .setInteger(longToSplitInt(value));
+    }
+  }
+
+  @VisibleForTesting
+  static String formatDuration(Duration duration) {
+    return DURATION_FORMATTER.print(duration.toPeriod());
+  }
+
+  private static final PeriodFormatter DURATION_FORMATTER =
+      new PeriodFormatterBuilder()
+          .appendDays()
+          .appendSuffix("d")
+          .minimumPrintedDigits(2)
+          .appendHours()
+          .appendSuffix("h")
+          .printZeroAlways()
+          .appendMinutes()
+          .appendSuffix("m")
+          .appendSeconds()
+          .appendSuffix("s")
+          .toFormatter();
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java
new file mode 100644
index 0000000..2fb55f7
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.runners.core.ElementByteSizeObservable;
+import org.apache.beam.runners.dataflow.worker.counters.Counter;
+import org.apache.beam.runners.dataflow.worker.counters.CounterFactory;
+import org.apache.beam.runners.dataflow.worker.counters.CounterName;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * A Dataflow-specific version of {@link ElementCounter}, which specifies the object counter name
+ * differently as PhysicalElementCount. Additionally, it counts element windows as ElementCount.
+ */
+public class DataflowOutputCounter implements ElementCounter {
+  /** Number of physical element and multiple-window assignments that were serialized/processed. */
+  private static final String OBJECT_COUNTER_NAME = "-PhysicalElementCount";
+  /** Number of logical element and single window pairs that were processed. */
+  private static final String ELEMENT_COUNTER_NAME = "-ElementCount";
+
+  private static final String MEAN_BYTE_COUNTER_NAME = "-MeanByteCount";
+
+  private OutputObjectAndByteCounter objectAndByteCounter;
+  private Counter<Long, ?> elementCount;
+
+  public DataflowOutputCounter(
+      String outputName, CounterFactory counterFactory, NameContext nameContext) {
+    this(outputName, null, counterFactory, nameContext);
+  }
+
+  public DataflowOutputCounter(
+      String outputName,
+      ElementByteSizeObservable<?> elementByteSizeObservable,
+      CounterFactory counterFactory,
+      NameContext nameContext) {
+    objectAndByteCounter =
+        new OutputObjectAndByteCounter(elementByteSizeObservable, counterFactory, nameContext);
+    objectAndByteCounter.countObject(outputName + OBJECT_COUNTER_NAME);
+    objectAndByteCounter.countMeanByte(outputName + MEAN_BYTE_COUNTER_NAME);
+    createElementCounter(counterFactory, nameContext, outputName + ELEMENT_COUNTER_NAME);
+  }
+
+  @Override
+  public void update(Object elem) throws Exception {
+    objectAndByteCounter.update(elem);
+    long windowsSize = ((WindowedValue<?>) elem).getWindows().size();
+    if (windowsSize == 0) {
+      // GroupingShuffleReader produces ValueInEmptyWindows.
+      // For now, we count the element at least once to keep the current counter
+      // behavior.
+      elementCount.addValue(1L);
+    } else {
+      elementCount.addValue(windowsSize);
+    }
+  }
+
+  @Override
+  public void finishLazyUpdate(Object elem) {
+    objectAndByteCounter.finishLazyUpdate(elem);
+  }
+
+  @VisibleForTesting
+  static String getElementCounterName(String prefix) {
+    return prefix + ELEMENT_COUNTER_NAME;
+  }
+
+  @VisibleForTesting
+  static String getObjectCounterName(String prefix) {
+    return prefix + OBJECT_COUNTER_NAME;
+  }
+
+  @VisibleForTesting
+  static String getMeanByteCounterName(String prefix) {
+    return prefix + MEAN_BYTE_COUNTER_NAME;
+  }
+
+  private void createElementCounter(CounterFactory factory, NameContext nameContext, String name) {
+    // TODO: use the name context to name the counter
+    elementCount = factory.longSum(CounterName.named(name));
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java
new file mode 100644
index 0000000..294630d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java
@@ -0,0 +1,177 @@
+/*
+ * 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.dataflow.worker;
+
+import java.util.Collection;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Materialization;
+import org.apache.beam.sdk.transforms.Materializations;
+import org.apache.beam.sdk.transforms.Materializations.MultimapView;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ViewFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+
+/**
+ * The minimal amount of information required to create and use a {@link SideInputReader} when using
+ * the portability framework within the Dataflow Runner harness.
+ *
+ * <p>Note that this is being used to satisfy type constraints for {@link SideInputReader} used
+ * throughout the Dataflow Runner harness and only supports returning the tag, the coder, the
+ * materialization, and the window coder. All other methods throw {@link
+ * UnsupportedOperationException}.
+ *
+ * <p>TODO: Migrate to a runner only specific concept of a side input to be used with {@link
+ * SideInputReader}s.
+ */
+public class DataflowPortabilityPCollectionView<K, V, W extends BoundedWindow>
+    implements PCollectionView<MultimapView<K, V>> {
+
+  public static <K, V> PCollectionView<MultimapView<K, V>> with(
+      TupleTag<KV<K, V>> tag, FullWindowedValueCoder<KV<K, V>> coder) {
+    return new DataflowPortabilityPCollectionView(tag, coder);
+  }
+
+  private final TupleTag<KV<K, V>> tag;
+  private final FullWindowedValueCoder<KV<K, V>> coder;
+
+  private DataflowPortabilityPCollectionView(
+      TupleTag<KV<K, V>> tag, FullWindowedValueCoder<KV<K, V>> coder) {
+    this.tag = tag;
+    this.coder = coder;
+  }
+
+  @Nullable
+  @Override
+  public PCollection<?> getPCollection() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleTag<KV<K, V>> getTagInternal() {
+    return tag;
+  }
+
+  @Override
+  public ViewFn<?, MultimapView<K, V>> getViewFn() {
+    return (ViewFn) PortabilityViewFn.INSTANCE;
+  }
+
+  /**
+   * A minimal type {@link ViewFn} that satisfies requirements to be used when executing portable
+   * pipelines.
+   */
+  public static class PortabilityViewFn<K, V>
+      extends ViewFn<MultimapView<K, V>, MultimapView<K, V>> {
+    private static final PortabilityViewFn<Object, Object> INSTANCE = new PortabilityViewFn<>();
+
+    // prevent instantiation
+    private PortabilityViewFn() {}
+
+    @Override
+    public Materialization<MultimapView<K, V>> getMaterialization() {
+      return Materializations.multimap();
+    }
+
+    @Override
+    public MultimapView<K, V> apply(MultimapView<K, V> o) {
+      return o;
+    }
+  };
+
+  @Override
+  public WindowMappingFn<?> getWindowMappingFn() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public WindowingStrategy<KV<K, V>, W> getWindowingStrategyInternal() {
+    return WindowingStrategy.of(
+        new WindowFn<KV<K, V>, W>() {
+          @Override
+          public Collection<W> assignWindows(AssignContext c) throws Exception {
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public void mergeWindows(MergeContext c) throws Exception {
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public boolean isCompatible(WindowFn<?, ?> other) {
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public Coder<W> windowCoder() {
+            return (Coder) coder.getWindowCoder();
+          }
+
+          @Override
+          public WindowMappingFn<W> getDefaultWindowMappingFn() {
+            throw new UnsupportedOperationException();
+          }
+        });
+  }
+
+  @Override
+  public Coder<?> getCoderInternal() {
+    return coder.getValueCoder();
+  }
+
+  @Override
+  public String getName() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Pipeline getPipeline() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Map<TupleTag<?>, PValue> expand() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void finishSpecifyingOutput(
+      String transformName, PInput input, PTransform<?, ?> transform) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void finishSpecifying(PInput upstreamInput, PTransform<?, ?> upstreamTransform) {
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java
new file mode 100644
index 0000000..d8b7cba
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java
@@ -0,0 +1,125 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn;
+
+import com.google.common.collect.Iterables;
+import java.util.Collection;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.StateNamespaces.WindowNamespace;
+import org.apache.beam.runners.core.TimerInternals.TimerData;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.joda.time.Instant;
+
+/**
+ * Runs a {@link ProcessFn} by constructing the appropriate contexts and passing them in.
+ *
+ * <p>This is the Dataflow-specific version of the runner-agnostic ProcessFnRunner. It's needed
+ * because the code for handling pushback on streaming side inputs in Dataflow is also divergent
+ * from the runner-agnostic code in runners-core. If that code is ever unified, so can this class.
+ */
+class DataflowProcessFnRunner<InputT, OutputT, RestrictionT>
+    implements DoFnRunner<KeyedWorkItem<String, KV<InputT, RestrictionT>>, OutputT> {
+  private final DoFnRunner<KeyedWorkItem<String, KV<InputT, RestrictionT>>, OutputT> simpleRunner;
+
+  DataflowProcessFnRunner(
+      DoFnRunner<KeyedWorkItem<String, KV<InputT, RestrictionT>>, OutputT> simpleRunner) {
+    this.simpleRunner = simpleRunner;
+  }
+
+  @Override
+  public void startBundle() {
+    simpleRunner.startBundle();
+  }
+
+  @Override
+  public void processElement(
+      WindowedValue<KeyedWorkItem<String, KV<InputT, RestrictionT>>> compressedElem) {
+    simpleRunner.processElement(placeIntoElementWindow(compressedElem));
+  }
+
+  private static <T> WindowedValue<KeyedWorkItem<String, T>> placeIntoElementWindow(
+      WindowedValue<KeyedWorkItem<String, T>> compressedElem) {
+    checkTrivialOuterWindows(compressedElem);
+    WindowedValue<KeyedWorkItem<String, T>> res =
+        WindowedValue.of(
+            compressedElem.getValue(),
+            BoundedWindow.TIMESTAMP_MIN_VALUE,
+            getUnderlyingWindow(compressedElem.getValue()),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    return res;
+  }
+
+  // TODO: move this and the next function into ProcessFn.
+  private static <T> void checkTrivialOuterWindows(
+      WindowedValue<KeyedWorkItem<String, T>> windowedKWI) {
+    // In practice it will be in 0 or 1 windows (ValueInEmptyWindows or ValueInGlobalWindow)
+    Collection<? extends BoundedWindow> outerWindows = windowedKWI.getWindows();
+    if (!outerWindows.isEmpty()) {
+      checkArgument(
+          outerWindows.size() == 1,
+          "The KeyedWorkItem itself must not be in multiple windows, but was in: %s",
+          outerWindows);
+      BoundedWindow onlyWindow = Iterables.getOnlyElement(outerWindows);
+      checkArgument(
+          onlyWindow instanceof GlobalWindow,
+          "KeyedWorkItem must be in the Global window, but was in: %s",
+          onlyWindow);
+    }
+  }
+
+  private static <T> BoundedWindow getUnderlyingWindow(KeyedWorkItem<String, T> kwi) {
+    if (Iterables.isEmpty(kwi.elementsIterable())) {
+      // ProcessFn sets only a single timer.
+      TimerData timer = Iterables.getOnlyElement(kwi.timersIterable());
+      return ((WindowNamespace) timer.getNamespace()).getWindow();
+    } else {
+      // KWI must have a single element in elementsIterable, because it follows a GBK by a
+      // uniquely generated key.
+      // Additionally, windows must be exploded before GBKIntoKeyedWorkItems, so there's also
+      // only a single window.
+      WindowedValue<T> value = Iterables.getOnlyElement(kwi.elementsIterable());
+      return Iterables.getOnlyElement(value.getWindows());
+    }
+  }
+
+  @Override
+  public void onTimer(
+      String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {
+    throw new UnsupportedOperationException("Unsupported for ProcessFn");
+  }
+
+  @Override
+  public void finishBundle() {
+    simpleRunner.finishBundle();
+  }
+
+  @Override
+  public DoFn<KeyedWorkItem<String, KV<InputT, RestrictionT>>, OutputT> getFn() {
+    return simpleRunner.getFn();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java
new file mode 100644
index 0000000..67b97fd
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java
@@ -0,0 +1,194 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.fn.BeamFnControlService;
+import org.apache.beam.runners.dataflow.worker.fn.ServerFactory;
+import org.apache.beam.runners.dataflow.worker.fn.data.BeamFnDataGrpcService;
+import org.apache.beam.runners.dataflow.worker.fn.logging.BeamFnLoggingService;
+import org.apache.beam.runners.dataflow.worker.fn.stream.ServerStreamObserverFactory;
+import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingInitializer;
+import org.apache.beam.runners.fnexecution.GrpcContextHeaderAccessorProvider;
+import org.apache.beam.runners.fnexecution.control.FnApiControlClient;
+import org.apache.beam.runners.fnexecution.state.GrpcStateService;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.vendor.grpc.v1.io.grpc.Server;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the harness for executing Dataflow jobs that make use of the Beam Fn API, and operates
+ * independently of the SDK(s) being used by users.
+ *
+ * <p>The worker harness is a mediator between Dataflow Service and the SDK, translating
+ * instructions (such as map tasks) from the Dataflow Service/DFE into Fn API instructions, and vice
+ * versa.
+ */
+public class DataflowRunnerHarness {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowRunnerHarness.class);
+
+  /** Fetches and processes work units from the Dataflow service. */
+  public static void main(String[] unusedArgs) throws Exception {
+    @Nullable RunnerApi.Pipeline pipeline = DataflowWorkerHarnessHelper.getPipelineFromEnv();
+
+    // This descriptor is used for all services except logging. They are isolated to keep
+    // critical traffic protected from best effort traffic.
+    ApiServiceDescriptor controlApiService = DataflowWorkerHarnessHelper.getControlDescriptor();
+    ApiServiceDescriptor loggingApiService = DataflowWorkerHarnessHelper.getLoggingDescriptor();
+
+    LOG.info(
+        "{} started, using port {} for control, {} for logging.",
+        DataflowRunnerHarness.class,
+        controlApiService,
+        loggingApiService);
+
+    DataflowWorkerHarnessHelper.initializeLogging(DataflowRunnerHarness.class);
+    DataflowWorkerHarnessOptions pipelineOptions =
+        DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions(
+            DataflowRunnerHarness.class);
+    DataflowWorkerHarnessHelper.configureLogging(pipelineOptions);
+
+    // Initialized registered file systems.˜
+    FileSystems.setDefaultPipelineOptions(pipelineOptions);
+
+    ServerFactory serverFactory = ServerFactory.fromOptions(pipelineOptions);
+    ServerStreamObserverFactory streamObserverFactory =
+        ServerStreamObserverFactory.fromOptions(pipelineOptions);
+
+    Server servicesServer = null;
+    Server loggingServer = null;
+    try (BeamFnLoggingService beamFnLoggingService =
+            new BeamFnLoggingService(
+                loggingApiService,
+                DataflowWorkerLoggingInitializer.getSdkLoggingHandler()::publish,
+                streamObserverFactory::from,
+                GrpcContextHeaderAccessorProvider.getHeaderAccessor());
+        BeamFnControlService beamFnControlService =
+            new BeamFnControlService(
+                controlApiService,
+                streamObserverFactory::from,
+                GrpcContextHeaderAccessorProvider.getHeaderAccessor());
+        BeamFnDataGrpcService beamFnDataService =
+            new BeamFnDataGrpcService(
+                pipelineOptions,
+                controlApiService,
+                streamObserverFactory::from,
+                GrpcContextHeaderAccessorProvider.getHeaderAccessor());
+        GrpcStateService beamFnStateService = GrpcStateService.create()) {
+
+      servicesServer =
+          serverFactory.create(
+              controlApiService,
+              ImmutableList.of(beamFnControlService, beamFnDataService, beamFnStateService));
+
+      loggingServer =
+          serverFactory.create(loggingApiService, ImmutableList.of(beamFnLoggingService));
+
+      start(
+          pipeline,
+          pipelineOptions,
+          beamFnControlService,
+          beamFnDataService,
+          controlApiService,
+          beamFnStateService);
+      servicesServer.shutdown();
+      loggingServer.shutdown();
+    } finally {
+      if (servicesServer != null) {
+        servicesServer.awaitTermination(30, TimeUnit.SECONDS);
+        servicesServer.shutdownNow();
+      }
+      if (loggingServer != null) {
+        loggingServer.awaitTermination(30, TimeUnit.SECONDS);
+        loggingServer.shutdownNow();
+      }
+    }
+  }
+
+  @SuppressWarnings("InfiniteLoopStatement")
+  public static void start(
+      @Nullable RunnerApi.Pipeline pipeline,
+      DataflowWorkerHarnessOptions pipelineOptions,
+      BeamFnControlService beamFnControlService,
+      BeamFnDataGrpcService beamFnDataService,
+      ApiServiceDescriptor stateApiServiceDescriptor,
+      GrpcStateService beamFnStateService)
+      throws Exception {
+
+    SdkHarnessRegistry sdkHarnessRegistry =
+        SdkHarnessRegistries.createFnApiSdkHarnessRegistry(
+            stateApiServiceDescriptor, beamFnStateService, beamFnDataService);
+    if (pipelineOptions.isStreaming()) {
+      DataflowWorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG);
+      LOG.info("Initializing Streaming Worker.");
+
+      StreamingDataflowWorker worker =
+          StreamingDataflowWorker.forStreamingFnWorkerHarness(
+              Collections.emptyList(), client, pipelineOptions, pipeline, sdkHarnessRegistry);
+      worker.startStatusPages();
+      worker.start();
+      ExecutorService executor = Executors.newSingleThreadExecutor();
+      executor.execute(
+          () -> { // Task to get new client connections
+            while (true) {
+              LOG.info("Waiting for client.");
+              // Never close controlClient. It will be closed  when the client terminates the
+              // connection.
+              FnApiControlClient controlClient = beamFnControlService.get();
+              LOG.info("Control client connected for {}", controlClient.getWorkerId());
+              controlClient.onClose(sdkHarnessRegistry::unregisterWorkerClient);
+              // register the client with worker once connected
+              sdkHarnessRegistry.registerWorkerClient(controlClient);
+            }
+          });
+      worker.waitTillExecutionFinishes();
+      executor.shutdownNow();
+    } else {
+      while (true) {
+        try (FnApiControlClient controlClient = beamFnControlService.get()) {
+          DataflowWorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG);
+          LOG.info("Initializing Batch Worker.");
+          BatchDataflowWorker worker =
+              BatchDataflowWorker.forBatchFnWorkerHarness(
+                  pipeline, sdkHarnessRegistry, client, pipelineOptions);
+          // register the client with worker once connected
+          sdkHarnessRegistry.registerWorkerClient(controlClient);
+          LOG.info("Client connected.");
+          try {
+            while (true) {
+              worker.getAndPerformWork();
+            }
+          } catch (IOException e) {
+            LOG.error("Getting and performing work failed.", e);
+          }
+        }
+      }
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java
new file mode 100644
index 0000000..f989e65
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java
@@ -0,0 +1,179 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.base.MoreObjects;
+import java.io.Closeable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.beam.runners.dataflow.worker.DataflowOperationContext.DataflowExecutionState;
+import org.apache.beam.runners.dataflow.worker.counters.Counter;
+import org.apache.beam.runners.dataflow.worker.counters.CounterName;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+
+/**
+ * This class tracks time and bytes spent when consuming side inputs.
+ *
+ * <p>This class is designed to track consumption of side inputs across fused steps.
+ *
+ * <p>We represent a side input as a declaring step, and an input index. The declaring step is the
+ * step that originally receives the side input for consumption, and the input index in which the
+ * declaring step receives the side input that we want to identify. The declaring step originally
+ * receives the side input, but it may not be the only step that spends time reading from this side
+ * input. Therefore, to represent the actual consumption of a side input, it is necessary to use
+ * three things: 1) the declaring step, 2) the input index, and 3) the currently executing step (as
+ * it may be different from the declaring step).
+ *
+ * <p>The following table summarizes the two different steps tracked when it comes to side inputs:
+ *
+ * <table>
+ *   <tr>
+ *     <th>Side Input Read Counter tracks two steps:</th>
+ *     <th>Declaring Step Name</th>
+ *     <th>Requesting Step Name</th>
+ *   </tr>
+ *   <tr>
+ *     <td>The value of these can vary because:</td>
+ *     <td>IsmReader instances are shared between multiple threads for the same PCollectionView</td>
+ *     <td>The same Lazy Iterable can be passed to and consumed by downstream steps as an output
+ *     element.</td>
+ *   </tr>
+ *   <tr>
+ *     <td>The current value is tracked by:</td>
+ *     <td>DelegatingIsmReader and DataflowSideInputReadCounter</td>
+ *     <td>The ExecutionStateTracker.</td>
+ *   </tr>
+ *   <tr>
+ *     <td>The current value is read by:</td>
+ *     <td>IsmReader and IsmPrefixReaderIterator</td>
+ *     <td>DataflowSideInputReadCounter, in the checkStep function.</td>
+ *   </tr>
+ * </table>
+ *
+ * <p>This is the case for both batch pipelines, and streaming pipelines, although the underlying
+ * storage layers are different (GCS for Batch, Windmill state for Streaming).
+ *
+ * <p>As an example of a pipeline where the declaring step and the consuming step of a side input
+ * are not the same, suppose a pipeline of the following form:
+ *
+ * <p>SideInputPCollection -> View.AsIterable ------------------ | v MainInputPCollection --->
+ * FirstParDo(...).withSideInput() -> AnotherParDo(...)
+ *
+ * <p>In this pipeline, the FirstParDo transform may simply emit the Iterable provided by the side
+ * input, and the AnotherParDo may be the one that actually manipulates that Iterable. This is
+ * possible because both ParDos will be fused, so they will simply exchange objects in memory.
+ */
+public class DataflowSideInputReadCounter implements SideInputReadCounter {
+  private final DataflowExecutionContext executionContext;
+
+  /** These two attributes describe the side input via a declaring step name, and an input index. */
+  private final DataflowOperationContext declaringOperationContext;
+
+  private final int sideInputIndex;
+
+  /** Maps containing the byte counters, and execution states associated to different steps. */
+  private final Map<NameContext, Counter<Long, Long>> byteCounters;
+
+  private final Map<NameContext, DataflowExecutionState> executionStates;
+
+  /**
+   * {@link Counter}, {@link DataflowExecutionState}, and {@link NameContext} associated to the
+   * latest step to have consumed the side input.
+   */
+  private Counter<Long, Long> currentCounter;
+
+  private DataflowExecutionState currentExecutionState;
+  private NameContext latestConsumingStepName;
+
+  public DataflowSideInputReadCounter(
+      DataflowExecutionContext executionContext,
+      DataflowOperationContext operationContext,
+      int sideInputIndex) {
+    this.executionContext = executionContext;
+    this.sideInputIndex = sideInputIndex;
+    this.declaringOperationContext = operationContext;
+    byteCounters = new HashMap<>();
+    executionStates = new HashMap<>();
+    checkState();
+  }
+
+  private void checkState() {
+    DataflowExecutionState currentState =
+        (DataflowExecutionState) executionContext.getExecutionStateTracker().getCurrentState();
+    if (currentState == null
+        || currentState.getStepName().originalName() == null
+        || Objects.equals(latestConsumingStepName, currentState.getStepName())) {
+      // In this case, the step has not changed, and we'll just return.
+      return;
+    }
+    if (!byteCounters.containsKey(currentState.getStepName())) {
+      byteCounters.put(
+          currentState.getStepName(),
+          executionContext
+              .getCounterFactory()
+              .longSum(
+                  CounterName.named("read-sideinput-byte-count")
+                      .withOriginalName(declaringOperationContext.nameContext())
+                      .withOrigin("SYSTEM")
+                      .withOriginalRequestingStepName(currentState.getStepName().originalName())
+                      .withInputIndex(sideInputIndex)));
+
+      executionStates.put(
+          currentState.getStepName(),
+          executionContext
+              .getExecutionStateRegistry()
+              .getIOState(
+                  declaringOperationContext.nameContext(),
+                  "read-sideinput",
+                  currentState.getStepName().originalName(),
+                  sideInputIndex,
+                  currentState.getMetricsContainer(),
+                  currentState.getProfileScope()));
+    }
+    currentCounter = byteCounters.get(currentState.getStepName());
+    currentExecutionState = executionStates.get(currentState.getStepName());
+    latestConsumingStepName = currentState.getStepName();
+  }
+
+  @Override
+  public void addBytesRead(long n) {
+    if (currentCounter != null) {
+      currentCounter.addValue(n);
+    }
+  }
+
+  @Override
+  public Closeable enter() {
+    checkState();
+    // Only update status from tracked thread to avoid race condition and inconsistent state updates
+    if (executionContext.getExecutionStateTracker().getTrackedThread() != Thread.currentThread()) {
+      return () -> {};
+    }
+    return executionContext.getExecutionStateTracker().enterState(currentExecutionState);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("sideInputIndex", sideInputIndex)
+        .add("declaringStep", declaringOperationContext.nameContext())
+        .toString();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java
new file mode 100644
index 0000000..109c07a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java
@@ -0,0 +1,86 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.base.Preconditions;
+import org.apache.beam.runners.dataflow.worker.counters.CounterName;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.sdk.metrics.MetricName;
+
+/** This holds system metrics related constants used in Batch and Streaming. */
+public class DataflowSystemMetrics {
+
+  public static final MetricName THROTTLING_MSECS_METRIC_NAME =
+      MetricName.named("dataflow-throttling-metrics", "throttling-msecs");
+
+  // TODO: Provide an utility in SDK 'ThrottlingReporter' to update throttling time.
+
+  /** System counters populated by streaming dataflow workers. */
+  public enum StreamingSystemCounterNames {
+    WINDMILL_SHUFFLE_BYTES_READ("WindmillShuffleBytesRead"),
+    WINDMILl_STATE_BYTES_READ("WindmillStateBytesRead"),
+    WINDMILl_STATE_BYTES_WRITTEN("WindmillStateBytesWritten"),
+    JAVA_HARNESS_USED_MEMORY("dataflow_java_harness_used_memory"),
+    JAVA_HARNESS_MAX_MEMORY("dataflow_java_harness_max_memory"),
+    JAVA_HARNESS_RESTARTS("dataflow_java_harness_restarts");
+
+    private final String name;
+
+    StreamingSystemCounterNames(String name) {
+      this.name = name;
+    }
+
+    public CounterName counterName() {
+      return CounterName.named(name);
+    }
+  }
+
+  /** System counters populated by streaming dataflow worker for each stage. */
+  public enum StreamingPerStageSystemCounterNames {
+
+    /**
+     * Total amount of time spent processing a stage, aggregated across all the concurrent tasks for
+     * a stage.
+     */
+    TOTAL_PROCESSING_MSECS("dataflow_total_processing_msecs"),
+
+    /**
+     * Total amount of time spent processing a stage, aggregated across all the concurrent tasks for
+     * a stage.
+     */
+    TIMER_PROCESSING_MSECS("dataflow_timer_processing_msecs"),
+
+    /**
+     * This is based on user updated metric "throttled-msecs", reported as part of system metrics so
+     * that streaming autoscaler can access it.
+     */
+    THROTTLED_MSECS("dataflow_throttled_msecs");
+
+    private final String namePrefix;
+
+    StreamingPerStageSystemCounterNames(String namePrefix) {
+      this.namePrefix = namePrefix;
+    }
+
+    public CounterName counterName(NameContext nameContext) {
+      Preconditions.checkNotNull(nameContext.systemName());
+      return CounterName.named(namePrefix + "-" + nameContext.systemName());
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkExecutor.java
new file mode 100644
index 0000000..e905c8f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkExecutor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkExecutor;
+
+/** An general executor for work in the Dataflow legacy and Beam harness. */
+public interface DataflowWorkExecutor extends WorkExecutor {
+
+  /**
+   * Extract metric updates that are still pending to be reported to the Dataflow service, removing
+   * them from this {@link DataflowWorkExecutor}.
+   */
+  Iterable<CounterUpdate> extractMetricUpdates();
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java
new file mode 100644
index 0000000..2dec434
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java
@@ -0,0 +1,121 @@
+/*
+ * 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.dataflow.worker;
+
+import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudDuration;
+import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime;
+
+import com.google.api.client.util.Clock;
+import com.google.api.services.dataflow.model.ApproximateSplitRequest;
+import com.google.api.services.dataflow.model.WorkItem;
+import com.google.api.services.dataflow.model.WorkItemServiceState;
+import com.google.common.annotations.VisibleForTesting;
+import java.util.concurrent.ScheduledExecutorService;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkExecutor;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkProgressUpdater;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataflowWorkProgressUpdater implements the WorkProgressUpdater interface for the Cloud Dataflow
+ * system.
+ */
+@NotThreadSafe
+public class DataflowWorkProgressUpdater extends WorkProgressUpdater {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkProgressUpdater.class);
+
+  private final WorkItemStatusClient workItemStatusClient;
+
+  /** The WorkItem for which work progress updates are sent. */
+  private final WorkItem workItem;
+
+  public DataflowWorkProgressUpdater(
+      WorkItemStatusClient workItemStatusClient, WorkItem workItem, WorkExecutor worker) {
+    super(worker, Integer.MAX_VALUE);
+    this.workItemStatusClient = workItemStatusClient;
+    this.workItem = workItem;
+  }
+
+  /**
+   * The {@link ScheduledExecutorService} parameter is used to inject a stubbed executor that uses
+   * virtual time for testing, and the {@link Clock} parameter is used to inject a mock clock that
+   * provides virtual time.
+   */
+  @VisibleForTesting
+  DataflowWorkProgressUpdater(
+      WorkItemStatusClient workItemStatusClient,
+      WorkItem workItem,
+      WorkExecutor worker,
+      ScheduledExecutorService executor,
+      Clock clock) {
+    super(worker, Integer.MAX_VALUE, executor, clock);
+    this.workItemStatusClient = workItemStatusClient;
+    this.workItem = workItem;
+  }
+
+  @Override
+  protected String workString() {
+    return workItemStatusClient.uniqueWorkId();
+  }
+
+  @Override
+  protected long getWorkUnitLeaseExpirationTimestamp() {
+    return getLeaseExpirationTimestamp(workItem);
+  }
+
+  @Override
+  protected long getWorkUnitSuggestedReportingInterval() {
+    return fromCloudDuration(workItem.getReportStatusInterval()).getMillis();
+  }
+
+  @Override
+  protected void reportProgressHelper() throws Exception {
+    WorkItemServiceState result =
+        workItemStatusClient.reportUpdate(
+            dynamicSplitResultToReport, Duration.millis(requestedLeaseDurationMs));
+    if (result != null) {
+      // Resets state after a successful progress report.
+      dynamicSplitResultToReport = null;
+
+      progressReportIntervalMs =
+          nextProgressReportInterval(
+              fromCloudDuration(result.getReportStatusInterval()).getMillis(),
+              leaseRemainingTime(getLeaseExpirationTimestamp(result)));
+
+      ApproximateSplitRequest suggestedStopPoint = result.getSplitRequest();
+      if (suggestedStopPoint != null) {
+        LOG.info("Proposing dynamic split of work unit {} at {}", workString(), suggestedStopPoint);
+        dynamicSplitResultToReport =
+            worker.requestDynamicSplit(
+                SourceTranslationUtils.toDynamicSplitRequest(suggestedStopPoint));
+      }
+    }
+  }
+
+  /** Returns the given work unit's lease expiration timestamp. */
+  private long getLeaseExpirationTimestamp(WorkItem workItem) {
+    return fromCloudTime(workItem.getLeaseExpireTime()).getMillis();
+  }
+
+  /** Returns the given work unit service state lease expiration timestamp. */
+  private long getLeaseExpirationTimestamp(WorkItemServiceState workItemServiceState) {
+    return fromCloudTime(workItemServiceState.getLeaseExpireTime()).getMillis();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java
new file mode 100644
index 0000000..6adb75b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java
@@ -0,0 +1,257 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.MoreObjects.firstNonNull;
+import static org.apache.beam.runners.dataflow.util.TimeUtil.toCloudDuration;
+import static org.apache.beam.runners.dataflow.util.TimeUtil.toCloudTime;
+import static org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames.CAPABILITY_REMOTE_SOURCE;
+import static org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames.WORK_ITEM_TYPE_MAP_TASK;
+import static org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames.WORK_ITEM_TYPE_REMOTE_SOURCE_TASK;
+import static org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames.WORK_ITEM_TYPE_SEQ_MAP_TASK;
+import static org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames.WORK_ITEM_TYPE_STREAMING_CONFIG_TASK;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.LeaseWorkItemRequest;
+import com.google.api.services.dataflow.model.LeaseWorkItemResponse;
+import com.google.api.services.dataflow.model.ReportWorkItemStatusRequest;
+import com.google.api.services.dataflow.model.ReportWorkItemStatusResponse;
+import com.google.api.services.dataflow.model.WorkItem;
+import com.google.api.services.dataflow.model.WorkItemServiceState;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.util.PropertyNames;
+import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkProgressUpdater;
+import org.apache.beam.sdk.util.Transport;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+
+/** A Dataflow WorkUnit client that fetches WorkItems from the Dataflow service. */
+@ThreadSafe
+class DataflowWorkUnitClient implements WorkUnitClient {
+  private final Logger logger;
+
+  /**
+   * Work items are reported as complete using this class's reportWorkItemStatus() method on the
+   * same thread that requested the item using getWorkItem(). This thread local variable is used to
+   * tag the current thread with the stage start time during getWorkItem() so that the elapsed
+   * execution time can be easily determined in reportWorkItemStatus(). A similar thread-local
+   * mechanism is used in DataflowWorkerLoggingMDC to track other metadata about the current
+   * operation being executed.
+   */
+  private static final ThreadLocal<DateTime> stageStartTime = new ThreadLocal<>();
+
+  private final CounterShortIdCache shortIdCache;
+  private final Dataflow dataflow;
+  private final DataflowWorkerHarnessOptions options;
+
+  /**
+   * Creates a client that fetches WorkItems from the Dataflow service.
+   *
+   * @param options The pipeline options.
+   */
+  DataflowWorkUnitClient(DataflowWorkerHarnessOptions options, Logger logger) {
+    this.dataflow = options.getDataflowClient();
+    this.options = options;
+    this.logger = logger;
+    this.shortIdCache = new CounterShortIdCache();
+  }
+
+  /**
+   * Gets a {@link WorkItem} from the Dataflow service, or returns {@link Optional#absent()} if no
+   * work was found.
+   *
+   * <p>If work is returned, the calling thread should call reportWorkItemStatus after completing it
+   * and before requesting another work item.
+   */
+  @Override
+  public Optional<WorkItem> getWorkItem() throws IOException {
+    List<String> workItemTypes =
+        ImmutableList.of(
+            WORK_ITEM_TYPE_MAP_TASK,
+            WORK_ITEM_TYPE_SEQ_MAP_TASK,
+            WORK_ITEM_TYPE_REMOTE_SOURCE_TASK);
+    // All remote sources require the "remote_source" capability. Dataflow's
+    // custom sources are further tagged with the format "custom_source".
+    List<String> capabilities =
+        ImmutableList.<String>of(
+            options.getWorkerId(), CAPABILITY_REMOTE_SOURCE, PropertyNames.CUSTOM_SOURCE_FORMAT);
+
+    Optional<WorkItem> workItem = getWorkItemInternal(workItemTypes, capabilities);
+    if (!workItem.isPresent() || workItem.get().getId() == null) {
+      logger.warn("Discarding invalid work item {}", workItem.orNull());
+      return Optional.absent();
+    }
+
+    WorkItem work = workItem.get();
+
+    final String stage;
+    if (work.getMapTask() != null) {
+      stage = work.getMapTask().getStageName();
+      logger.info("Starting MapTask stage {}", stage);
+    } else if (work.getSeqMapTask() != null) {
+      stage = work.getSeqMapTask().getStageName();
+      logger.info("Starting SeqMapTask stage {}", stage);
+    } else if (work.getSourceOperationTask() != null) {
+      stage = work.getSourceOperationTask().getStageName();
+      logger.info("Starting SourceOperationTask stage {}", stage);
+    } else {
+      stage = null;
+    }
+    DataflowWorkerLoggingMDC.setStageName(stage);
+
+    stageStartTime.set(DateTime.now());
+    DataflowWorkerLoggingMDC.setWorkId(Long.toString(work.getId()));
+
+    return workItem;
+  }
+
+  /**
+   * Gets a global streaming config {@link WorkItem} from the Dataflow service, or returns {@link
+   * Optional#absent()} if no work was found.
+   */
+  @Override
+  public Optional<WorkItem> getGlobalStreamingConfigWorkItem() throws IOException {
+    return getWorkItemInternal(
+        ImmutableList.of(WORK_ITEM_TYPE_STREAMING_CONFIG_TASK), ImmutableList.of());
+  }
+
+  /**
+   * Gets a streaming config {@link WorkItem} for the given computation from the Dataflow service,
+   * or returns {@link Optional#absent()} if no work was found.
+   */
+  @Override
+  public Optional<WorkItem> getStreamingConfigWorkItem(String computationId) throws IOException {
+    Preconditions.checkNotNull(computationId);
+    return getWorkItemInternal(
+        ImmutableList.of("streaming_config_task:" + computationId), ImmutableList.of());
+  }
+
+  private Optional<WorkItem> getWorkItemInternal(
+      List<String> workItemTypes, List<String> capabilities) throws IOException {
+    LeaseWorkItemRequest request = new LeaseWorkItemRequest();
+    request.setFactory(Transport.getJsonFactory());
+    request.setWorkItemTypes(workItemTypes);
+    request.setWorkerCapabilities(capabilities);
+    request.setWorkerId(options.getWorkerId());
+    request.setCurrentWorkerTime(toCloudTime(DateTime.now()));
+
+    // This shouldn't be necessary, but a valid cloud duration string is
+    // required by the Google API parsing framework.  TODO: Fix the framework
+    // so that an empty or not-present string can be used as a default value.
+    request.setRequestedLeaseDuration(
+        toCloudDuration(Duration.millis(WorkProgressUpdater.DEFAULT_LEASE_DURATION_MILLIS)));
+
+    logger.debug("Leasing work: {}", request);
+
+    LeaseWorkItemResponse response =
+        dataflow
+            .projects()
+            .locations()
+            .jobs()
+            .workItems()
+            .lease(options.getProject(), options.getRegion(), options.getJobId(), request)
+            .execute();
+    logger.debug("Lease work response: {}", response);
+
+    List<WorkItem> workItems = response.getWorkItems();
+    if (workItems == null || workItems.isEmpty()) {
+      // We didn't lease any work.
+      return Optional.absent();
+    } else if (workItems.size() > 1) {
+      throw new IOException(
+          "This version of the SDK expects no more than one work item from the service: "
+              + response);
+    }
+    WorkItem work = response.getWorkItems().get(0);
+
+    // Looks like the work's a'ight.
+    return Optional.of(work);
+  }
+
+  /** Reports the status of the most recently requested work item. */
+  @Override
+  public WorkItemServiceState reportWorkItemStatus(WorkItemStatus workItemStatus)
+      throws IOException {
+    DateTime endTime = DateTime.now();
+    workItemStatus.setFactory(Transport.getJsonFactory());
+    logger.debug("Reporting work status: {}", workItemStatus);
+    // Log the stage execution time of finished stages that have a stage name.  This will not be set
+    // in the event this status is associated with a dummy work item.
+    if (firstNonNull(workItemStatus.getCompleted(), Boolean.FALSE)
+        && DataflowWorkerLoggingMDC.getStageName() != null) {
+      DateTime startTime = stageStartTime.get();
+      if (startTime != null) {
+        // This thread should have been tagged with the stage start time during getWorkItem(),
+        Interval elapsed = new Interval(startTime, endTime);
+        int numErrors = workItemStatus.getErrors() == null ? 0 : workItemStatus.getErrors().size();
+        logger.info(
+            "Finished processing stage {} with {} errors in {} seconds ",
+            DataflowWorkerLoggingMDC.getStageName(),
+            numErrors,
+            (double) elapsed.toDurationMillis() / 1000);
+      }
+    }
+    shortIdCache.shortenIdsIfAvailable(workItemStatus.getCounterUpdates());
+    ReportWorkItemStatusRequest request =
+        new ReportWorkItemStatusRequest()
+            .setWorkerId(options.getWorkerId())
+            .setWorkItemStatuses(Collections.singletonList(workItemStatus))
+            .setCurrentWorkerTime(toCloudTime(endTime));
+    ReportWorkItemStatusResponse result =
+        dataflow
+            .projects()
+            .locations()
+            .jobs()
+            .workItems()
+            .reportStatus(options.getProject(), options.getRegion(), options.getJobId(), request)
+            .execute();
+    if (result == null) {
+      logger.warn("Report work item status response: {}", result);
+      throw new IOException("Got null work item status response");
+    }
+
+    if (result.getWorkItemServiceStates() == null) {
+      logger.warn("Report work item status response: {}", result);
+      throw new IOException("Report work item status contained no work item service states");
+    }
+    if (result.getWorkItemServiceStates().size() != 1) {
+      logger.warn("Report work item status response: {}", result);
+      throw new IOException(
+          "This version of the SDK expects exactly one work item service state from the service "
+              + "but got "
+              + result.getWorkItemServiceStates().size()
+              + " states");
+    }
+    shortIdCache.storeNewShortIds(request, result);
+    WorkItemServiceState state = result.getWorkItemServiceStates().get(0);
+    logger.debug("ReportWorkItemStatus result: {}", state);
+    return state;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
new file mode 100644
index 0000000..cd6fd1d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
@@ -0,0 +1,122 @@
+/*
+ * 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.dataflow.worker;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.Security;
+import java.util.logging.Handler;
+import java.util.logging.LogManager;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingInitializer;
+import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.vendor.protobuf.v3.com.google.protobuf.TextFormat;
+import org.conscrypt.OpenSSLProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A helper class for initialization of the Dataflow worker harness. */
+public final class DataflowWorkerHarnessHelper {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkerHarnessHelper.class);
+
+  private static final String CONTROL_API_SERVICE_DESCRIPTOR = "CONTROL_API_SERVICE_DESCRIPTOR";
+  private static final String LOGGING_API_SERVICE_DESCRIPTOR = "LOGGING_API_SERVICE_DESCRIPTOR";
+  private static final String ROOT_LOGGER_NAME = "";
+  private static final String PIPELINE_PATH = "PIPELINE_PATH";
+
+  public static DataflowWorkerHarnessOptions initializeGlobalStateAndPipelineOptions(
+      Class<?> workerHarnessClass) throws Exception {
+    /* Enable fast SSL provider. */
+    Security.insertProviderAt(new OpenSSLProvider(), 1);
+
+    /* Extract pipeline options. */
+    DataflowWorkerHarnessOptions pipelineOptions =
+        WorkerPipelineOptionsFactory.createFromSystemProperties();
+    pipelineOptions.setAppName(workerHarnessClass.getSimpleName());
+
+    /* Configure logging with job-specific properties. */
+    DataflowWorkerLoggingMDC.setJobId(pipelineOptions.getJobId());
+    DataflowWorkerLoggingMDC.setWorkerId(pipelineOptions.getWorkerId());
+
+    return pipelineOptions;
+  }
+
+  public static void initializeLogging(Class<?> workerHarnessClass) {
+    /* Set up exception handling tied to the workerHarnessClass. */
+    Thread.setDefaultUncaughtExceptionHandler(
+        new WorkerUncaughtExceptionHandler(LoggerFactory.getLogger(workerHarnessClass)));
+
+    // Reset the global log manager, get the root logger and remove the default log handlers.
+    LogManager logManager = LogManager.getLogManager();
+    logManager.reset();
+    java.util.logging.Logger rootLogger = logManager.getLogger(ROOT_LOGGER_NAME);
+    for (Handler handler : rootLogger.getHandlers()) {
+      rootLogger.removeHandler(handler);
+    }
+    DataflowWorkerLoggingInitializer.initialize();
+  }
+
+  public static void configureLogging(DataflowWorkerHarnessOptions pipelineOptions) {
+
+    DataflowWorkerLoggingInitializer.configure(pipelineOptions);
+  }
+
+  public static Endpoints.ApiServiceDescriptor parseApiServiceDescriptorFromText(
+      String descriptorText) throws TextFormat.ParseException {
+    Endpoints.ApiServiceDescriptor.Builder apiServiceDescriptorBuilder =
+        Endpoints.ApiServiceDescriptor.newBuilder();
+    TextFormat.merge(descriptorText, apiServiceDescriptorBuilder);
+    return apiServiceDescriptorBuilder.build();
+  }
+
+  public static Endpoints.ApiServiceDescriptor getLoggingDescriptor()
+      throws TextFormat.ParseException {
+    return parseApiServiceDescriptorFromText(System.getenv().get(LOGGING_API_SERVICE_DESCRIPTOR));
+  }
+
+  public static Endpoints.ApiServiceDescriptor getControlDescriptor()
+      throws TextFormat.ParseException {
+    return parseApiServiceDescriptorFromText(System.getenv().get(CONTROL_API_SERVICE_DESCRIPTOR));
+  }
+
+  // TODO: make env logic private to main() so it is never done outside of initializing the process
+  public static @Nullable RunnerApi.Pipeline getPipelineFromEnv() throws IOException {
+    String pipelinePath = System.getenv(PIPELINE_PATH);
+    if (pipelinePath == null) {
+      LOG.warn("Missing pipeline environment variable '%s'", PIPELINE_PATH);
+      return null;
+    }
+
+    File pipelineFile = new File(System.getenv(PIPELINE_PATH));
+    if (!pipelineFile.exists()) {
+      LOG.warn("Pipeline path '%s' does not exist", pipelineFile);
+      return null;
+    }
+
+    try (FileInputStream inputStream = new FileInputStream(pipelineFile)) {
+      RunnerApi.Pipeline pipelineProto = RunnerApi.Pipeline.parseFrom(inputStream);
+      LOG.info("Found portable pipeline:\n{}", TextFormat.printToString(pipelineProto));
+      return pipelineProto;
+    }
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactory.java
new file mode 100644
index 0000000..5f0b7cf
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.collect.ImmutableMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * A factory that dispatches to all known factories in the Dataflow SDK based on the value of {@link
+ * CloudObject#getClassName()} for the specified {@code DoFn}.
+ */
+public class DefaultParDoFnFactory implements ParDoFnFactory {
+  private final ImmutableMap<String, ParDoFnFactory> defaultFactories;
+
+  public DefaultParDoFnFactory() {
+    defaultFactories =
+        ImmutableMap.<String, ParDoFnFactory>builder()
+            .put("DoFn", UserParDoFnFactory.createDefault())
+            .put("CombineValuesFn", new CombineValuesFnFactory())
+            .put("MergeBucketsDoFn", new GroupAlsoByWindowParDoFnFactory())
+            .put("AssignBucketsDoFn", new AssignWindowsParDoFnFactory())
+            .put("MergeWindowsDoFn", new GroupAlsoByWindowParDoFnFactory())
+            .put("AssignWindowsDoFn", new AssignWindowsParDoFnFactory())
+            .put("ReifyTimestampAndWindowsDoFn", new ReifyTimestampAndWindowsParDoFnFactory())
+            .put("SplittableProcessFn", SplittableProcessFnFactory.createDefault())
+            .put("CreateIsmShardKeyAndSortKeyDoFn", new CreateIsmShardKeyAndSortKeyDoFnFactory())
+            .put("ToIsmRecordForMultimapDoFn", new ToIsmRecordForMultimapDoFnFactory())
+            .put("ValuesDoFn", new ValuesDoFnFactory())
+            .put("PairWithConstantKeyDoFn", new PairWithConstantKeyDoFnFactory())
+            .put(
+                "StreamingPCollectionViewWriterDoFn",
+                new StreamingPCollectionViewWriterDoFnFactory())
+            .build();
+  }
+
+  @Override
+  public ParDoFn create(
+      PipelineOptions options,
+      CloudObject cloudUserFn,
+      List<SideInputInfo> sideInputInfos,
+      TupleTag<?> mainOutputTag,
+      Map<TupleTag<?>, Integer> outputTupleTagsToReceiverIndices,
+      DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+
+    String className = cloudUserFn.getClassName();
+    ParDoFnFactory factory = defaultFactories.get(className);
+
+    if (factory == null) {
+      throw new Exception("No known ParDoFnFactory for " + className);
+    }
+
+    return factory.create(
+        options,
+        cloudUserFn,
+        sideInputInfos,
+        mainOutputTag,
+        outputTupleTagsToReceiverIndices,
+        executionContext,
+        operationContext);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCell.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCell.java
new file mode 100644
index 0000000..80b0ac0
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCell.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.dataflow.worker;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.beam.runners.core.metrics.CounterCell;
+import org.apache.beam.runners.core.metrics.DirtyState;
+import org.apache.beam.runners.core.metrics.MetricCell;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.MetricName;
+
+/**
+ * Version of {@link CounterCell} supporting multi-thread safe mutations and extraction of delta
+ * values.
+ */
+public class DeltaCounterCell implements Counter, MetricCell<Long> {
+
+  private final MetricName name;
+  private final AtomicLong value = new AtomicLong();
+
+  public DeltaCounterCell(MetricName name) {
+    this.name = name;
+  }
+
+  @Override
+  public void inc(long n) {
+    value.addAndGet(n);
+  }
+
+  @Override
+  public void inc() {
+    inc(1);
+  }
+
+  @Override
+  public void dec() {
+    inc(-1);
+  }
+
+  @Override
+  public void dec(long n) {
+    inc(-1 * n);
+  }
+
+  @Override
+  public MetricName getName() {
+    return name;
+  }
+
+  @Override
+  public DirtyState getDirty() {
+    throw new UnsupportedOperationException(
+        String.format("%s doesn't support the getDirty", getClass().getSimpleName()));
+  }
+
+  @Override
+  public Long getCumulative() {
+    throw new UnsupportedOperationException("getCumulative is not supported by Streaming Metrics");
+  }
+
+  public Long getSum() {
+    return value.get();
+  }
+
+  public Long getSumAndReset() {
+    return value.getAndSet(0);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DeltaDistributionCell.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DeltaDistributionCell.java
new file mode 100644
index 0000000..44daa4c
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DeltaDistributionCell.java
@@ -0,0 +1,75 @@
+/*
+ * 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.dataflow.worker;
+
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.beam.runners.core.metrics.DirtyState;
+import org.apache.beam.runners.core.metrics.DistributionCell;
+import org.apache.beam.runners.core.metrics.DistributionData;
+import org.apache.beam.runners.core.metrics.MetricCell;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.MetricName;
+
+/**
+ * A version of {@link DistributionCell} that supports extracting the delta and clearing it out.
+ *
+ * <p>TODO: Modify the Beam DistributionCell to support extracting the delta.
+ */
+public class DeltaDistributionCell implements Distribution, MetricCell<DistributionData> {
+  private final AtomicReference<DistributionData> value =
+      new AtomicReference<>(DistributionData.EMPTY);
+  private final MetricName name;
+
+  public DeltaDistributionCell(MetricName name) {
+    this.name = name;
+  }
+
+  /** Increment the distribution by the given amount. */
+  @Override
+  public void update(long n) {
+    update(DistributionData.singleton(n));
+  }
+
+  void update(DistributionData data) {
+    DistributionData original;
+    do {
+      original = value.get();
+    } while (!value.compareAndSet(original, original.combine(data)));
+  }
+
+  @Override
+  public DirtyState getDirty() {
+    throw new UnsupportedOperationException(
+        String.format("%s doesn't support the getDirty", getClass().getSimpleName()));
+  }
+
+  @Override
+  public DistributionData getCumulative() {
+    throw new UnsupportedOperationException("getCumulative is not supported by Streaming Metrics");
+  }
+
+  public DistributionData getAndReset() {
+    return value.getAndUpdate((unused) -> DistributionData.EMPTY);
+  }
+
+  @Override
+  public MetricName getName() {
+    return name;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManager.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManager.java
new file mode 100644
index 0000000..f841a84
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManager.java
@@ -0,0 +1,47 @@
+/*
+ * 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.dataflow.worker;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.DoFnInfo;
+
+/** Contains methods for obtaining and clearing {@link DoFnInfo} instances. t */
+public interface DoFnInstanceManager {
+  /**
+   * Get the {@link DoFnInfo} contained by this {@link DoFnInstanceManager} without obtaining
+   * ownership of that {@link DoFnInfo}. {@link DoFn} processing methods should not be called on the
+   * {@link DoFn} contained within the {@link DoFnInfo} returned by this call.
+   */
+  DoFnInfo<?, ?> peek() throws Exception;
+
+  /**
+   * Get the {@link DoFnInfo} contained by this {@link DoFnInstanceManager}, and obtain ownership of
+   * the returned {@link DoFnInfo}.
+   */
+  DoFnInfo<?, ?> get() throws Exception;
+
+  /**
+   * Relinquish ownership of the provided {@link DoFnInfo} after successfully completing a bundle.
+   */
+  void complete(DoFnInfo<?, ?> fnInfo) throws Exception;
+
+  /** Relinquish ownership of the provided {@link DoFnInfo} after aborting a bundle. */
+  void abort(@Nullable DoFnInfo<?, ?> fnInfo) throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java
new file mode 100644
index 0000000..6f81edb
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java
@@ -0,0 +1,123 @@
+/*
+ * 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.dataflow.worker;
+
+import java.util.concurrent.ConcurrentLinkedQueue;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.util.DoFnInfo;
+import org.apache.beam.sdk.util.SerializableUtils;
+
+/** Common {@link DoFnInstanceManager} implementations. */
+public class DoFnInstanceManagers {
+  /**
+   * Returns a {@link DoFnInstanceManager} that returns {@link DoFnInfo} instances obtained by
+   * deserializing the provided bytes. {@link DoFnInstanceManager} will call {@link DoFn.Setup} as
+   * required before returning the {@link DoFnInfo}, and {@link DoFn.Teardown} as appropriate.
+   */
+  public static DoFnInstanceManager cloningPool(DoFnInfo<?, ?> info) {
+    return new ConcurrentQueueInstanceManager(info);
+  }
+
+  /**
+   * Returns a {@link DoFnInstanceManager} that always returns the provided {@link DoFnInfo} from
+   * calls to {@link DoFnInstanceManager#get()}.
+   *
+   * <p>Note that the returned {@link DoFnInstanceManager} will not call {@link DoFn.Setup} or
+   * {@link DoFn.Teardown} under any circumstances, and will reuse the provided instance across all
+   * threads, even if {@link DoFnInstanceManager#abort(DoFnInfo)} is called.
+   */
+  public static DoFnInstanceManager singleInstance(DoFnInfo<?, ?> info) {
+    return new SingleInstanceManager(info);
+  }
+
+  private static class ConcurrentQueueInstanceManager implements DoFnInstanceManager {
+    private final byte[] serializedFnInfo;
+    private final ConcurrentLinkedQueue<DoFnInfo<?, ?>> fns;
+
+    private ConcurrentQueueInstanceManager(DoFnInfo<?, ?> info) {
+      this.serializedFnInfo = SerializableUtils.serializeToByteArray(info);
+      fns = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public DoFnInfo<?, ?> peek() throws Exception {
+      DoFnInfo<?, ?> fn = fns.peek();
+      if (fn == null) {
+        fn = deserializeCopy();
+        fns.offer(fn);
+      }
+      return fn;
+    }
+
+    @Override
+    public DoFnInfo<?, ?> get() throws Exception {
+      DoFnInfo<?, ?> fn = fns.poll();
+      if (fn == null) {
+        fn = deserializeCopy();
+      }
+      return fn;
+    }
+
+    private DoFnInfo<?, ?> deserializeCopy() throws Exception {
+      DoFnInfo<?, ?> fn;
+      fn = (DoFnInfo<?, ?>) SerializableUtils.deserializeFromByteArray(serializedFnInfo, null);
+      DoFnInvokers.invokerFor(fn.getDoFn()).invokeSetup();
+      return fn;
+    }
+
+    @Override
+    public void complete(DoFnInfo<?, ?> fnInfo) throws Exception {
+      if (fnInfo != null) {
+        fns.offer(fnInfo);
+      }
+    }
+
+    @Override
+    public void abort(DoFnInfo<?, ?> fnInfo) throws Exception {
+      if (fnInfo != null && fnInfo.getDoFn() != null) {
+        DoFnInvokers.invokerFor(fnInfo.getDoFn()).invokeTeardown();
+      }
+    }
+  }
+
+  private static class SingleInstanceManager implements DoFnInstanceManager {
+    private final DoFnInfo<?, ?> info;
+
+    private SingleInstanceManager(DoFnInfo<?, ?> info) {
+      this.info = info;
+    }
+
+    @Override
+    public DoFnInfo<?, ?> peek() throws Exception {
+      return info;
+    }
+
+    @Override
+    public DoFnInfo<?, ?> get() throws Exception {
+      return info;
+    }
+
+    @Override
+    public void complete(DoFnInfo<?, ?> fnInfo) throws Exception {}
+
+    @Override
+    public void abort(DoFnInfo<?, ?> fnInfo) throws Exception {}
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnRunnerFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnRunnerFactory.java
new file mode 100644
index 0000000..05ab598
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnRunnerFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.dataflow.worker;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+
+interface DoFnRunnerFactory<InputT, OutputT> {
+  DoFnRunner<InputT, OutputT> createRunner(
+      DoFn<InputT, OutputT> fn,
+      PipelineOptions options,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      Iterable<PCollectionView<?>> sideInputViews,
+      SideInputReader sideInputReader,
+      Coder<InputT> inputCoder,
+      Map<TupleTag<?>, Coder<?>> outputCoders,
+      WindowingStrategy<?, ?> windowingStrategy,
+      DataflowExecutionContext.DataflowStepContext stepContext,
+      DataflowExecutionContext.DataflowStepContext userStepContext,
+      OutputManager outputManager);
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExecutionStateKey.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExecutionStateKey.java
new file mode 100644
index 0000000..b508598
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExecutionStateKey.java
@@ -0,0 +1,87 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Ordering;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+
+/**
+ * Execution states are uniquely identified by the step name, the state name, and for states
+ * representing IO operations, also their requesting step name, and input index.
+ */
+@AutoValue
+public abstract class ExecutionStateKey implements Comparable<ExecutionStateKey> {
+
+  /** Return the {@link NameContext} identifying the step associated with this key. */
+  public abstract NameContext getStepName();
+
+  /** Return the activity within the step this key represents. */
+  public abstract String getStateName();
+
+  /**
+   * Return the step that is associated to some IO operation, such as reading of Shuffle or side
+   * inputs.
+   */
+  @Nullable
+  public abstract String getRequestingStepName();
+
+  /**
+   * Return the collection index that is associated to some operation, such as reading side inputs.
+   */
+  @Nullable
+  public abstract Integer getInputIndex();
+
+  /**
+   * Create a new {@link ExecutionStateKey} representing the given step and activity.
+   *
+   * @param nameContext names describing the step associated with this state key.
+   * @param stateName additional string identifying the activity within that step.
+   * @param requestingStepName for states representing IO activity, this is the currently-running
+   *     step name.
+   * @param inputIndex for states representing IO activities related to a PCollection, this is the
+   *     index of the PCollection that they represent. This is mainly used to represent side inputs
+   *     (e.g. index 1, 2, etc).
+   */
+  public static ExecutionStateKey create(
+      NameContext nameContext,
+      String stateName,
+      @Nullable String requestingStepName,
+      @Nullable Integer inputIndex) {
+    return new AutoValue_ExecutionStateKey(nameContext, stateName, requestingStepName, inputIndex);
+  }
+
+  @Override
+  public int compareTo(ExecutionStateKey o) {
+    return ComparisonChain.start()
+        .compare(getStateName(), o.getStateName())
+        .compare(
+            getStepName().originalName(),
+            o.getStepName().originalName(),
+            Ordering.natural().nullsFirst())
+        .compare(
+            getStepName().stageName(), o.getStepName().stageName(), Ordering.natural().nullsFirst())
+        .compare(
+            getRequestingStepName(), o.getRequestingStepName(), Ordering.natural().nullsFirst())
+        .compare(getInputIndex(), o.getInputIndex(), Ordering.natural().nullsFirst())
+        .result();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExecutionStateRegistry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExecutionStateRegistry.java
new file mode 100644
index 0000000..7256b13
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExecutionStateRegistry.java
@@ -0,0 +1,115 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Predicates.notNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.common.collect.FluentIterable;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.worker.counters.NameContext;
+import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ExecutionStateTracker.ExecutionState;
+import org.apache.beam.sdk.metrics.MetricsContainer;
+
+/** Manages the instances of {@link ExecutionState} */
+public abstract class ExecutionStateRegistry {
+
+  /**
+   * Generally, the execution states should be created when the MapTask is created, so this doesn't
+   * need to be concurrently accessed. In fact, it could be immutable after the creation of the
+   * MapTask. But, since it is theoretically possible for new states to be created, and the
+   * execution sampler may be reading this at the same time, we use a concurrent map for safety.
+   */
+  private final Map<ExecutionStateKey, DataflowOperationContext.DataflowExecutionState>
+      createdStates = new ConcurrentSkipListMap<>();
+
+  /**
+   * Get an existing state or create a {@link DataflowOperationContext.DataflowExecutionState} that
+   * represents time spent within a step.
+   */
+  public DataflowOperationContext.DataflowExecutionState getState(
+      final NameContext nameContext,
+      final String stateName,
+      final MetricsContainer container,
+      final ProfileScope profileScope) {
+    return getStateInternal(nameContext, stateName, null, null, container, profileScope);
+  }
+
+  /**
+   * Get an existing state or create a {@link DataflowOperationContext.DataflowExecutionState} that
+   * represents the consumption of some kind of IO, such as reading of Side Input, or Shuffle data.
+   *
+   * <p>An IO-related ExcecutionState may represent: * A Side Input collection as declaringStep +
+   * inputIndex. The consumption of the side input is represented by (declaringStep, inputIndex,
+   * requestingStepName), where requestingStepName is the step that causes the IO to occur. * A
+   * Shuffle IO as the GBK step for that shuffle. The consumption of the side input is represented
+   * by (declaringStep, requestingStepName), where requestingStepName is the step that causes the IO
+   * to occur.
+   */
+  public DataflowOperationContext.DataflowExecutionState getIOState(
+      final NameContext nameContext,
+      final String stateName,
+      final String requestingStepName,
+      @Nullable final Integer inputIndex,
+      @Nullable final MetricsContainer container,
+      final ProfileScope profileScope) {
+    return getStateInternal(
+        nameContext, stateName, requestingStepName, inputIndex, container, profileScope);
+  }
+
+  private DataflowOperationContext.DataflowExecutionState getStateInternal(
+      final NameContext nameContext,
+      final String stateName,
+      @Nullable final String requestingStepName,
+      @Nullable final Integer inputIndex,
+      @Nullable final MetricsContainer container,
+      final ProfileScope profileScope) {
+    ExecutionStateKey stateKey =
+        ExecutionStateKey.create(nameContext, stateName, requestingStepName, inputIndex);
+    return createdStates.computeIfAbsent(
+        stateKey,
+        (unused) ->
+            createState(
+                nameContext, stateName, requestingStepName, inputIndex, container, profileScope));
+  }
+
+  /**
+   * Internal method to create and register an ExecutionState.
+   *
+   * <p>Do not call this method directly. Instead, use the {@link ExecutionStateRegistry#getState},
+   * and {@link ExecutionStateRegistry#getIOState} public methods, or implement your own public
+   * create method.
+   */
+  protected abstract DataflowOperationContext.DataflowExecutionState createState(
+      NameContext nameContext,
+      String stateName,
+      @Nullable String requestingStepName,
+      @Nullable Integer inputIndex,
+      @Nullable MetricsContainer container,
+      ProfileScope profileScope);
+
+  public Iterable<CounterUpdate> extractUpdates(boolean isFinalUpdate) {
+    return FluentIterable.from(createdStates.values())
+        .transform((state) -> state.extractUpdate(isFinalUpdate))
+        .filter(notNull());
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java
new file mode 100644
index 0000000..f8e8b59
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.dataflow.worker;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Sets;
+import java.util.EnumSet;
+import java.util.Set;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * A convenient class to provide fast lookup of enabled experiments in the worker code.
+ *
+ * <p>To add a new experiment, update the {@link Experiment} enum.
+ *
+ * <p>To determine if an experiment is enabled, instantiate an {@link ExperimentContext} with the
+ * {@link PipelineOptions} and call {@link #isEnabled} to test if it is enabled.
+ */
+public class ExperimentContext {
+
+  /** Enumeration of all known experiments. */
+  public enum Experiment {
+    IntertransformIO("intertransform_io"), // Intertransform metrics for Shuffle IO (insights)
+    SideInputIOMetrics("sideinput_io_metrics"); // Intertransform metrics for Side Input IO
+
+    private final String name;
+
+    Experiment(String name) {
+      this.name = name;
+    }
+
+    public String getName() {
+      return this.name;
+    }
+
+    @Override
+    public String toString() {
+      return this.name;
+    }
+  }
+
+  private final EnumSet<Experiment> enabledExperiments = EnumSet.noneOf(Experiment.class);
+
+  /**
+   * Create an {@link ExperimentContext} from the given {@link PipelineOptions}.
+   *
+   * <p>NOTE: This will parse the experiment strings in the given options, so it should not be
+   * re-parsed frequently.
+   */
+  public static ExperimentContext parseFrom(PipelineOptions options) {
+    return new ExperimentContext(options.as(DataflowPipelineDebugOptions.class).getExperiments());
+  }
+
+  @VisibleForTesting
+  static ExperimentContext parseFrom(Iterable<String> experimentNames) {
+    return new ExperimentContext(experimentNames);
+  }
+
+  private ExperimentContext(Iterable<String> experimentNames) {
+    if (experimentNames == null) {
+      return;
+    }
+    Set<String> strings = Sets.newHashSet(experimentNames);
+    for (Experiment experiment : Experiment.values()) {
+      if (strings.contains(experiment.getName())) {
+        enabledExperiments.add(experiment);
+      }
+    }
+  }
+
+  public boolean isEnabled(Experiment exp) {
+    return enabledExperiments.contains(exp);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
new file mode 100644
index 0000000..32e939e
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
@@ -0,0 +1,272 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.collect.ImmutableList;
+import java.io.Closeable;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputReceiver;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ReceivingOperation;
+import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
+import org.apache.beam.runners.fnexecution.data.FnDataService;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.state.BagState;
+import org.apache.beam.sdk.transforms.Materializations;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ViewFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+
+/**
+ * This {@link ReceivingOperation} is responsible for fetching any ready side inputs and also
+ * filtering any input elements that aren't ready by pushing them back into state.
+ */
+public class FetchAndFilterStreamingSideInputsOperation<T, W extends BoundedWindow>
+    extends ReceivingOperation {
+
+  private final StreamingSideInputFetcher<T, W> sideInputFetcher;
+  private final BlockingQueue<Object> elementsToProcess;
+  private final ExecutorService singleThreadExecutor;
+  private Future<Void> elementProcessor;
+  private static final Object POISON_PILL = new Object();
+
+  public FetchAndFilterStreamingSideInputsOperation(
+      OutputReceiver[] receivers,
+      DataflowOperationContext context,
+      InstructionRequestHandler instructionRequestHandler,
+      FnDataService beamFnDataService,
+      ApiServiceDescriptor dataServiceApiServiceDescriptor,
+      Supplier<String> idGenerator,
+      Coder<WindowedValue<T>> inputCoder,
+      WindowingStrategy<?, W> windowingStrategy,
+      DataflowExecutionContext.DataflowStepContext stepContext,
+      Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec> pCollectionViewToWindowMappingFns) {
+    super(receivers, context);
+
+    this.sideInputFetcher =
+        new StreamingSideInputFetcher<>(
+            buildPCollectionViewsWithSdkSupportedWindowMappingFn(
+                idGenerator,
+                instructionRequestHandler,
+                beamFnDataService,
+                dataServiceApiServiceDescriptor,
+                ((FullWindowedValueCoder) inputCoder).getWindowCoder(),
+                pCollectionViewToWindowMappingFns),
+            ((WindowedValueCoder) inputCoder).getValueCoder(),
+            windowingStrategy,
+            (StreamingModeExecutionContext.StreamingModeStepContext)
+                stepContext.namespacedToUser());
+    this.elementsToProcess = new LinkedBlockingQueue<>();
+    this.singleThreadExecutor = Executors.newSingleThreadExecutor();
+  }
+
+  /** A {@link PCollectionView} which forwards all calls to its delegate. */
+  private static class ForwardingPCollectionView<T> implements PCollectionView<T> {
+    private final PCollectionView<T> delegate;
+
+    private ForwardingPCollectionView(PCollectionView<T> delegate) {
+      this.delegate = delegate;
+    }
+
+    @Nullable
+    @Override
+    public PCollection<?> getPCollection() {
+      return delegate.getPCollection();
+    }
+
+    @Override
+    public TupleTag<?> getTagInternal() {
+      return delegate.getTagInternal();
+    }
+
+    @Override
+    public ViewFn<?, T> getViewFn() {
+      return delegate.getViewFn();
+    }
+
+    @Override
+    public WindowMappingFn<?> getWindowMappingFn() {
+      return delegate.getWindowMappingFn();
+    }
+
+    @Override
+    public WindowingStrategy<?, ?> getWindowingStrategyInternal() {
+      return delegate.getWindowingStrategyInternal();
+    }
+
+    @Override
+    public Coder<?> getCoderInternal() {
+      return delegate.getCoderInternal();
+    }
+
+    @Override
+    public String getName() {
+      return delegate.getName();
+    }
+
+    @Override
+    public Pipeline getPipeline() {
+      return delegate.getPipeline();
+    }
+
+    @Override
+    public Map<TupleTag<?>, PValue> expand() {
+      return delegate.expand();
+    }
+
+    @Override
+    public void finishSpecifyingOutput(
+        String transformName, PInput input, PTransform<?, ?> transform) {
+      delegate.finishSpecifyingOutput(transformName, input, transform);
+    }
+
+    @Override
+    public void finishSpecifying(PInput upstreamInput, PTransform<?, ?> upstreamTransform) {
+      delegate.finishSpecifying(upstreamInput, upstreamTransform);
+    }
+  }
+
+  private Iterable<PCollectionView<?>> buildPCollectionViewsWithSdkSupportedWindowMappingFn(
+      Supplier<String> idGenerator,
+      InstructionRequestHandler instructionRequestHandler,
+      FnDataService beamFnDataService,
+      ApiServiceDescriptor dataServiceApiServiceDescriptor,
+      Coder<BoundedWindow> mainInputWindowCoder,
+      Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec> pCollectionViewsToWindowMappingFns) {
+    ImmutableList.Builder<PCollectionView<?>> wrappedViews = ImmutableList.builder();
+    for (Map.Entry<PCollectionView<?>, RunnerApi.SdkFunctionSpec> entry :
+        pCollectionViewsToWindowMappingFns.entrySet()) {
+      WindowMappingFn windowMappingFn =
+          new FnApiWindowMappingFn(
+              idGenerator,
+              instructionRequestHandler,
+              dataServiceApiServiceDescriptor,
+              beamFnDataService,
+              entry.getValue(),
+              mainInputWindowCoder,
+              entry.getKey().getWindowingStrategyInternal().getWindowFn().windowCoder());
+      wrappedViews.add(
+          new ForwardingPCollectionView<Materializations.MultimapView>(
+              (PCollectionView) entry.getKey()) {
+            @Override
+            public WindowMappingFn<?> getWindowMappingFn() {
+              return windowMappingFn;
+            }
+          });
+    }
+    return wrappedViews.build();
+  }
+
+  @Override
+  public void start() throws Exception {
+    try (Closeable scope = context.enterStart()) {
+      super.start();
+
+      elementProcessor =
+          singleThreadExecutor.submit(
+              () -> {
+                Object compressedElem;
+                while ((compressedElem = elementsToProcess.take()) != POISON_PILL) {
+                  for (WindowedValue<T> elem :
+                      ((WindowedValue<T>) compressedElem).explodeWindows()) {
+                    if (!sideInputFetcher.storeIfBlocked(elem)) {
+                      outputToReceivers(elem);
+                    }
+                  }
+                }
+                return null;
+              });
+
+      // Find the set of ready windows.
+      Set<W> readyWindows = sideInputFetcher.getReadyWindows();
+
+      Iterable<BagState<WindowedValue<T>>> elementsBags =
+          sideInputFetcher.prefetchElements(readyWindows);
+
+      // Output any elements that have all their side inputs ready.
+      for (BagState<WindowedValue<T>> elementsBag : elementsBags) {
+        Iterable<WindowedValue<T>> elements = elementsBag.read();
+        for (WindowedValue<T> elem : elements) {
+          outputToReceivers(elem);
+        }
+        elementsBag.clear();
+      }
+      sideInputFetcher.releaseBlockedWindows(readyWindows);
+    }
+  }
+
+  @Override
+  public void process(Object compressedElement) throws Exception {
+    elementsToProcess.add(compressedElement);
+  }
+
+  @Override
+  public void finish() throws Exception {
+    try (Closeable scope = context.enterFinish()) {
+      // Add the poison pill so that the working thread finishes.
+      elementsToProcess.add(POISON_PILL);
+      elementProcessor.get();
+
+      // Remove all processed elements and add all blocked elements.
+      sideInputFetcher.persist();
+
+      super.finish();
+    }
+  }
+
+  @Override
+  public void abort() throws Exception {
+    try (Closeable scope = context.enterAbort()) {
+      super.abort();
+      // Cancel the element processor
+      elementProcessor.cancel(true);
+    }
+  }
+
+  public void outputToReceivers(WindowedValue<T> elem) throws Exception {
+    for (OutputReceiver receiver : receivers) {
+      receiver.process(elem);
+    }
+  }
+
+  @Override
+  public boolean supportsRestart() {
+    return true;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java
new file mode 100644
index 0000000..15023ba
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java
@@ -0,0 +1,82 @@
+/*
+ * 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.dataflow.worker;
+
+import com.google.common.collect.ImmutableList;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/** Utilities for handling filepatterns. */
+public class Filepatterns {
+  private static final Pattern AT_N_SPEC = Pattern.compile("@(?<N>\\d+)");
+
+  /**
+   * Expands the filepattern containing an {@code @N} wildcard.
+   *
+   * <p>Returns N filenames with the wildcard replaced with a string of the form {@code
+   * 0000i-of-0000N}. For example, for "gs://bucket/file@2.ism", returns an iterable of two elements
+   * "gs://bucket/file-00000-of-00002.ism" and "gs://bucket/file-00001-of-00002.ism".
+   *
+   * <p>The sequence number and N are formatted with the same number of digits (prepended by zeros).
+   * with at least 5 digits. N must be smaller than 1 billion.
+   *
+   * <p>If the filepattern contains no wildcards, returns the filepattern unchanged.
+   *
+   * @throws IllegalArgumentException if more than one wildcard is detected.
+   */
+  public static Iterable<String> expandAtNFilepattern(String filepattern) {
+    ImmutableList.Builder<String> builder = ImmutableList.builder();
+    Matcher match = AT_N_SPEC.matcher(filepattern);
+    if (!match.find()) {
+      builder.add(filepattern);
+    } else {
+      int numShards = Integer.parseInt(match.group("N"));
+      String formatString = "-%0" + getShardWidth(numShards, filepattern) + "d-of-%05d";
+      for (int i = 0; i < numShards; ++i) {
+        builder.add(
+            AT_N_SPEC.matcher(filepattern).replaceAll(String.format(formatString, i, numShards)));
+      }
+      if (match.find()) {
+        throw new IllegalArgumentException(
+            "More than one @N wildcard found in filepattern: " + filepattern);
+      }
+    }
+    return builder.build();
+  }
+
+  private static int getShardWidth(int numShards, String filepattern) {
+    if (numShards < 100000) {
+      return 5;
+    }
+    if (numShards < 1000000) {
+      return 6;
+    }
+    if (numShards < 10000000) {
+      return 7;
+    }
+    if (numShards < 100000000) {
+      return 8;
+    }
+    if (numShards < 1000000000) {
+      return 9;
+    }
+    throw new IllegalArgumentException(
+        "Unsupported number of shards: " + numShards + " in filepattern: " + filepattern);
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
new file mode 100644
index 0000000..8383234
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
@@ -0,0 +1,330 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.base.Strings;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import java.io.IOException;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.RegisterRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+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.SdkFunctionSpec;
+import org.apache.beam.runners.core.construction.CoderTranslation;
+import org.apache.beam.runners.core.construction.RehydratedComponents;
+import org.apache.beam.runners.core.construction.SdkComponents;
+import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
+import org.apache.beam.runners.fnexecution.data.FnDataService;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.fn.data.CloseableFnDataReceiver;
+import org.apache.beam.sdk.fn.data.InboundDataClient;
+import org.apache.beam.sdk.fn.data.LogicalEndpoint;
+import org.apache.beam.sdk.fn.data.RemoteGrpcPortRead;
+import org.apache.beam.sdk.fn.data.RemoteGrpcPortWrite;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.util.MoreFutures;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maps main input windows onto side input windows.
+ *
+ * <p>Note that this {@link WindowMappingFn} performs blocking calls over the data plane. The thread
+ * which maps side input windows should not be on the same thread that is reading from the same data
+ * plane channel.
+ *
+ * <p>TODO: Swap this with an implementation which streams all required window mappings per bundle
+ * instead of per mapping request. This requires rewriting the {@link StreamingSideInputFetcher} to
+ * not be inline calls and process elements over a stream.
+ */
+class FnApiWindowMappingFn<TargetWindowT extends BoundedWindow>
+    extends WindowMappingFn<TargetWindowT> {
+  private static final Logger LOG = LoggerFactory.getLogger(FnApiWindowMappingFn.class);
+  private static final byte[] EMPTY_ARRAY = new byte[0];
+
+  @AutoValue
+  public abstract static class CacheKey {
+    public static CacheKey create(SdkFunctionSpec windowMappingFn, BoundedWindow mainWindow) {
+      return new AutoValue_FnApiWindowMappingFn_CacheKey(windowMappingFn, mainWindow);
+    }
+
+    public abstract SdkFunctionSpec getWindowMappingFn();
+
+    public abstract BoundedWindow getMainWindow();
+  }
+
+  private static final Cache<CacheKey, BoundedWindow> sideInputMappingCache =
+      CacheBuilder.newBuilder().maximumSize(1000).build();
+
+  private final Supplier<String> idGenerator;
+  private final FnDataService beamFnDataService;
+  private final InstructionRequestHandler instructionRequestHandler;
+  private final SdkFunctionSpec windowMappingFn;
+  private final Coder<WindowedValue<KV<byte[], BoundedWindow>>> outboundCoder;
+  private final Coder<WindowedValue<KV<byte[], TargetWindowT>>> inboundCoder;
+  private final ProcessBundleDescriptor processBundleDescriptor;
+
+  FnApiWindowMappingFn(
+      Supplier<String> idGenerator,
+      InstructionRequestHandler instructionRequestHandler,
+      ApiServiceDescriptor dataServiceApiServiceDescriptor,
+      FnDataService beamFnDataService,
+      SdkFunctionSpec windowMappingFn,
+      Coder<BoundedWindow> mainInputWindowCoder,
+      Coder<TargetWindowT> sideInputWindowCoder) {
+    this.idGenerator = idGenerator;
+    this.instructionRequestHandler = instructionRequestHandler;
+    this.beamFnDataService = beamFnDataService;
+    this.windowMappingFn = windowMappingFn;
+
+    SdkComponents sdkComponents = SdkComponents.create();
+    sdkComponents.registerEnvironment(RunnerApi.Environment.getDefaultInstance());
+    String mainInputWindowCoderId;
+    String sideInputWindowCoderId;
+    String windowingStrategyId;
+    RunnerApi.Components components;
+    try {
+      mainInputWindowCoderId =
+          sdkComponents.registerCoder(
+              WindowedValue.getFullCoder(
+                  KvCoder.of(ByteArrayCoder.of(), mainInputWindowCoder),
+                  GlobalWindow.Coder.INSTANCE));
+      sideInputWindowCoderId =
+          sdkComponents.registerCoder(
+              WindowedValue.getFullCoder(
+                  KvCoder.of(ByteArrayCoder.of(), sideInputWindowCoder),
+                  GlobalWindow.Coder.INSTANCE));
+      windowingStrategyId =
+          sdkComponents.registerWindowingStrategy(WindowingStrategy.globalDefault());
+      RehydratedComponents rehydratedComponents =
+          RehydratedComponents.forComponents(sdkComponents.toComponents());
+      components = sdkComponents.toComponents();
+      outboundCoder =
+          (Coder)
+              CoderTranslation.fromProto(
+                  components.getCodersOrThrow(mainInputWindowCoderId), rehydratedComponents);
+      inboundCoder =
+          (Coder)
+              CoderTranslation.fromProto(
+                  components.getCodersOrThrow(sideInputWindowCoderId), rehydratedComponents);
+    } catch (IOException e) {
+      throw new IllegalStateException(
+          "Unable to create side input window mapping process bundle specification.", e);
+    }
+
+    processBundleDescriptor =
+        ProcessBundleDescriptor.newBuilder()
+            .putAllCoders(components.getCodersMap())
+            .putPcollections(
+                "inPC",
+                PCollection.newBuilder()
+                    .setCoderId(mainInputWindowCoderId)
+                    .setWindowingStrategyId(windowingStrategyId)
+                    .build())
+            .putPcollections(
+                "outPC",
+                PCollection.newBuilder()
+                    .setCoderId(sideInputWindowCoderId)
+                    .setWindowingStrategyId(windowingStrategyId)
+                    .build())
+            .putTransforms(
+                "read",
+                RunnerApi.PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn(RemoteGrpcPortRead.URN)
+                            .setPayload(
+                                RemoteGrpcPort.newBuilder()
+                                    .setApiServiceDescriptor(dataServiceApiServiceDescriptor)
+                                    .setCoderId(mainInputWindowCoderId)
+                                    .build()
+                                    .toByteString()))
+                    .putOutputs("in", "inPC")
+                    .build())
+            .putTransforms(
+                "map",
+                RunnerApi.PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn("beam:transform:map_windows:v1")
+                            .setPayload(windowMappingFn.toByteString()))
+                    .putInputs("in", "inPC")
+                    .putOutputs("out", "outPC")
+                    .build())
+            .putTransforms(
+                "write",
+                RunnerApi.PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn(RemoteGrpcPortWrite.URN)
+                            .setPayload(
+                                RemoteGrpcPort.newBuilder()
+                                    .setApiServiceDescriptor(dataServiceApiServiceDescriptor)
+                                    .setCoderId(sideInputWindowCoderId)
+                                    .build()
+                                    .toByteString()))
+                    .putInputs("out", "outPC")
+                    .build())
+            .putAllWindowingStrategies(components.getWindowingStrategiesMap())
+            .build();
+  }
+
+  @Override
+  public TargetWindowT getSideInputWindow(BoundedWindow mainWindow) {
+    try {
+      TargetWindowT rval =
+          (TargetWindowT)
+              sideInputMappingCache.get(
+                  CacheKey.create(windowMappingFn, mainWindow),
+                  () -> loadIfNeeded(windowMappingFn, mainWindow));
+      return rval;
+    } catch (ExecutionException e) {
+      throw new IllegalStateException(
+          String.format("Unable to load side input window for %s", mainWindow), e);
+    }
+  }
+
+  private TargetWindowT loadIfNeeded(SdkFunctionSpec windowMappingFn, BoundedWindow mainWindow) {
+    try {
+      String processRequestInstructionId = idGenerator.get();
+      InstructionRequest processRequest =
+          InstructionRequest.newBuilder()
+              .setInstructionId(processRequestInstructionId)
+              .setProcessBundle(
+                  ProcessBundleRequest.newBuilder()
+                      .setProcessBundleDescriptorReference(registerIfRequired()))
+              .build();
+
+      ConcurrentLinkedQueue<WindowedValue<KV<byte[], TargetWindowT>>> outputValue =
+          new ConcurrentLinkedQueue<>();
+
+      // Open the inbound consumer
+      InboundDataClient waitForInboundTermination =
+          beamFnDataService.receive(
+              LogicalEndpoint.of(
+                  processRequestInstructionId,
+                  BeamFnApi.Target.newBuilder()
+                      .setName("out")
+                      .setPrimitiveTransformReference("write")
+                      .build()),
+              inboundCoder,
+              outputValue::add);
+
+      CompletionStage<InstructionResponse> processResponse =
+          instructionRequestHandler.handle(processRequest);
+
+      // Open the outbound consumer
+      try (CloseableFnDataReceiver<WindowedValue<KV<byte[], BoundedWindow>>> outboundConsumer =
+          beamFnDataService.send(
+              LogicalEndpoint.of(
+                  processRequestInstructionId,
+                  BeamFnApi.Target.newBuilder()
+                      .setName("in")
+                      .setPrimitiveTransformReference("read")
+                      .build()),
+              outboundCoder)) {
+
+        outboundConsumer.accept(WindowedValue.valueInGlobalWindow(KV.of(EMPTY_ARRAY, mainWindow)));
+      }
+
+      // Check to see if processing the request failed.
+      throwIfFailure(processResponse);
+
+      waitForInboundTermination.awaitCompletion();
+      WindowedValue<KV<byte[], TargetWindowT>> sideInputWindow = outputValue.poll();
+      checkState(
+          sideInputWindow != null
+              && sideInputWindow.getValue() != null
+              && sideInputWindow.getValue().getValue() != null,
+          "Expected side input window to have been emitted by SDK harness.");
+      checkState(
+          outputValue.isEmpty(),
+          "Expected only a single side input window to have been emitted by "
+              + "the SDK harness but also received %s",
+          outputValue);
+      return sideInputWindow.getValue().getValue();
+    } catch (Exception e) {
+      LOG.error("Unable to map main input window {} to side input window.", mainWindow, e);
+      throw new IllegalStateException(e);
+    }
+  }
+
+  /** Should only be accessed from within {@link #registerIfRequired}. */
+  private String processBundleDescriptorId;
+
+  /**
+   * Register a process bundle descriptor for this remote window mapping fn.
+   *
+   * <p>Caches the result after the first registration.
+   */
+  private synchronized String registerIfRequired() throws ExecutionException, InterruptedException {
+    if (processBundleDescriptorId == null) {
+      String descriptorId = idGenerator.get();
+
+      CompletionStage<InstructionResponse> response =
+          instructionRequestHandler.handle(
+              InstructionRequest.newBuilder()
+                  .setInstructionId(idGenerator.get())
+                  .setRegister(
+                      RegisterRequest.newBuilder()
+                          .addProcessBundleDescriptor(
+                              processBundleDescriptor.toBuilder().setId(descriptorId).build())
+                          .build())
+                  .build());
+      throwIfFailure(response);
+      processBundleDescriptorId = descriptorId;
+    }
+    return processBundleDescriptorId;
+  }
+
+  private static InstructionResponse throwIfFailure(
+      CompletionStage<InstructionResponse> responseFuture)
+      throws ExecutionException, InterruptedException {
+    InstructionResponse response = MoreFutures.get(responseFuture);
+    if (!Strings.isNullOrEmpty(response.getError())) {
+      throw new IllegalStateException(
+          String.format(
+              "Client failed to process %s with error [%s].",
+              response.getInstructionId(), response.getError()));
+    }
+    return response;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java
new file mode 100644
index 0000000..ac4a8e5
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.dataflow.worker;
+
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver;
+
+/**
+ * A base class for {@link ParDoFn} implementations for overriding particular methods while
+ * forwarding the rest to an existing instance.
+ */
+public abstract class ForwardingParDoFn implements ParDoFn {
+
+  private ParDoFn delegate;
+
+  protected ForwardingParDoFn(ParDoFn delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void startBundle(Receiver... receivers) throws Exception {
+    delegate.startBundle(receivers);
+  }
+
+  @Override
+  public void processElement(Object elem) throws Exception {
+    delegate.processElement(elem);
+  }
+
+  @Override
+  public void processTimers() throws Exception {
+    delegate.processTimers();
+  }
+
+  @Override
+  public void finishBundle() throws Exception {
+    delegate.finishBundle();
+  }
+
+  @Override
+  public void abort() throws Exception {
+    delegate.abort();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFn.java
new file mode 100644
index 0000000..8c46702
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFn.java
@@ -0,0 +1,40 @@
+/*
+ * 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.dataflow.worker;
+
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
+import org.apache.beam.runners.core.OutputWindowedValue;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.core.StepContext;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * An interface for functions that execute {@link GroupAlsoByWindow}-like machinery and need access
+ * to worker internals, such as {@link TimerInternals} and {@link OutputWindowedValue}.
+ */
+public abstract class GroupAlsoByWindowFn<InputT, OutputT> {
+  /** Processes one input element. */
+  public abstract void processElement(
+      InputT element,
+      PipelineOptions options,
+      StepContext stepContext,
+      SideInputReader sideInputReader,
+      OutputWindowedValue<OutputT> output)
+      throws Exception;
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java
new file mode 100644
index 0000000..81e7fa4
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java
@@ -0,0 +1,132 @@
+/*
+ * 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.dataflow.worker;
+
+import java.util.Collection;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.OutputWindowedValue;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.core.StepContext;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+
+/**
+ * Runs a {@link GroupAlsoByWindowFn} by constructing the appropriate contexts and passing them in.
+ *
+ * @param <InputT> the type of the {@link GroupAlsoByWindowFn} (main) input elements
+ * @param <OutputT> the type of the {@link GroupAlsoByWindowFn} (main) output elements
+ */
+public class GroupAlsoByWindowFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+
+  private final PipelineOptions options;
+  /** The {@link GroupAlsoByWindowFn} being run. */
+  private final GroupAlsoByWindowFn<InputT, OutputT> fn;
+
+  private final SideInputReader sideInputReader;
+  private final OutputManager outputManager;
+  private final TupleTag<OutputT> mainOutputTag;
+  private final StepContext stepContext;
+
+  public GroupAlsoByWindowFnRunner(
+      PipelineOptions options,
+      GroupAlsoByWindowFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      StepContext stepContext) {
+    this.options = options;
+    this.fn = fn;
+    this.sideInputReader = sideInputReader;
+    this.outputManager = outputManager;
+    this.mainOutputTag = mainOutputTag;
+    this.stepContext = stepContext;
+  }
+
+  @Override
+  public void startBundle() {}
+
+  @Override
+  public void processElement(WindowedValue<InputT> elem) {
+    if (elem.getWindows().size() <= 1 || sideInputReader.isEmpty()) {
+      invokeProcessElement(elem);
+    } else {
+      // We could modify the windowed value (and the processContext) to
+      // avoid repeated allocations, but this is more straightforward.
+      for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
+        invokeProcessElement(windowedValue);
+      }
+    }
+  }
+
+  @Override
+  public void onTimer(
+      String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {
+    throw new UnsupportedOperationException(
+        String.format("Timers are not supported by %s", GroupAlsoByWindowFn.class.getSimpleName()));
+  }
+
+  private void invokeProcessElement(WindowedValue<InputT> elem) {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      OutputWindowedValue<OutputT> output =
+          new OutputWindowedValue<OutputT>() {
+            @Override
+            public void outputWindowedValue(
+                OutputT output,
+                Instant timestamp,
+                Collection<? extends BoundedWindow> windows,
+                PaneInfo pane) {
+              WindowedValue<OutputT> windowed = WindowedValue.of(output, timestamp, windows, pane);
+              outputManager.output(mainOutputTag, windowed);
+            }
+
+            @Override
+            public <AdditionalOutputT> void outputWindowedValue(
+                TupleTag<AdditionalOutputT> tag,
+                AdditionalOutputT output,
+                Instant timestamp,
+                Collection<? extends BoundedWindow> windows,
+                PaneInfo pane) {
+              throw new UnsupportedOperationException();
+            }
+          };
+      fn.processElement(elem.getValue(), options, stepContext, sideInputReader, output);
+    } catch (Exception ex) {
+      if (ex instanceof RuntimeException) {
+        throw (RuntimeException) ex;
+      }
+
+      throw new RuntimeException(ex);
+    }
+  }
+
+  @Override
+  public void finishBundle() {}
+
+  @Override
+  public DoFn<InputT, OutputT> getFn() {
+    return null;
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java
new file mode 100644
index 0000000..628e0c2
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java
@@ -0,0 +1,383 @@
+/*
+ * 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.dataflow.worker;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.runners.dataflow.util.Structs.getBytes;
+import static org.apache.beam.runners.dataflow.util.Structs.getObject;
+import static org.apache.beam.runners.dataflow.util.Structs.getString;
+
+import com.google.api.services.dataflow.model.SideInputInfo;
+import com.google.common.collect.Iterables;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents.RootCase;
+import org.apache.beam.runners.core.NullSideInputReader;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.runners.core.StateInternalsFactory;
+import org.apache.beam.runners.core.construction.RehydratedComponents;
+import org.apache.beam.runners.core.construction.WindowingStrategyTranslation;
+import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.util.CloudObject;
+import org.apache.beam.runners.dataflow.util.CloudObjects;
+import org.apache.beam.runners.dataflow.util.PropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.BatchGroupAlsoByWindowsDoFns;
+import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.beam.vendor.protobuf.v3.com.google.protobuf.InvalidProtocolBufferException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link ParDoFnFactory} to create GroupAlsoByWindowsDoFn instances according to specifications
+ * from the Dataflow service.
+ */
+class GroupAlsoByWindowParDoFnFactory implements ParDoFnFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GroupAlsoByWindowParDoFnFactory.class);
+
+  @Override
+  public ParDoFn create(
+      PipelineOptions options,
+      CloudObject cloudUserFn,
+      @Nullable List<SideInputInfo> sideInputInfos,
+      TupleTag<?> mainOutputTag,
+      Map<TupleTag<?>, Integer> outputTupleTagsToReceiverIndices,
+      final DataflowExecutionContext<?> executionContext,
+      DataflowOperationContext operationContext)
+      throws Exception {
+    Map.Entry<TupleTag<?>, Integer> entry =
+        Iterables.getOnlyElement(outputTupleTagsToReceiverIndices.entrySet());
+    checkArgument(
+        entry.getKey().equals(mainOutputTag),
+        "Output tags should reference only the main output tag: %s vs %s",
+        entry.getKey(),
+        mainOutputTag);
+    checkArgument(
+        entry.getValue() == 0,
+        "There should be a single receiver, but using receiver index %s",
+        entry.getValue());
+
+    byte[] encodedWindowingStrategy = getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN);
+    WindowingStrategy windowingStrategy;
+    try {
+      windowingStrategy = deserializeWindowingStrategy(encodedWindowingStrategy);
+    } catch (Exception e) {
+      // Temporarily choose default windowing strategy if fn API is enabled.
+      // TODO: Catch block disappears, becoming an error once Python SDK is compliant.
+      if (DataflowRunner.hasExperiment(
+          options.as(DataflowPipelineDebugOptions.class), "beam_fn_api")) {
+        LOG.info("FnAPI: Unable to deserialize windowing strategy, assuming default", e);
+        windowingStrategy = WindowingStrategy.globalDefault();
+      } else {
+        throw e;
+      }
+    }
+
+    byte[] serializedCombineFn = getBytes(cloudUserFn, WorkerPropertyNames.COMBINE_FN, null);
+    AppliedCombineFn<?, ?, ?, ?> combineFn = null;
+    if (serializedCombineFn != null) {
+      Object combineFnObj =
+          SerializableUtils.deserializeFromByteArray(serializedCombineFn, "serialized combine fn");
+      checkArgument(
+          combineFnObj instanceof AppliedCombineFn,
+          "unexpected kind of AppliedCombineFn: " + combineFnObj.getClass().getName());
+      combineFn = (AppliedCombineFn<?, ?, ?, ?>) combineFnObj;
+    }
+
+    Map<String, Object> inputCoderObject = getObject(cloudUserFn, WorkerPropertyNames.INPUT_CODER);
+
+    Coder<?> inputCoder = CloudObjects.coderFromCloudObject(CloudObject.fromSpec(inputCoderObject));
+    checkArgument(
+        inputCoder instanceof WindowedValueCoder,
+        "Expected WindowedValueCoder for inputCoder, got: " + inputCoder.getClass().getName());
+    @SuppressWarnings("unchecked")
+    WindowedValueCoder<?> windowedValueCoder = (WindowedValueCoder<?>) inputCoder;
+
+    Coder<?> elemCoder = windowedValueCoder.getValueCoder();
+    checkArgument(
+        elemCoder instanceof KvCoder,
+        "Expected KvCoder for inputCoder, got: " + elemCoder.getClass().getName());
+    @SuppressWarnings("unchecked")
+    KvCoder<?, ?> kvCoder = (KvCoder<?, ?>) elemCoder;
+
+    boolean isStreamingPipeline = options.as(StreamingOptions.class).isStreaming();
+
+    SideInputReader sideInputReader = NullSideInputReader.empty();
+    @Nullable AppliedCombineFn<?, ?, ?, ?> maybeMergingCombineFn = null;
+    if (combineFn != null) {
+      sideInputReader =
+          executionContext.getSideInputReader(
+              sideInputInfos, combineFn.getSideInputViews(), operationContext);
+
+      String phase = getString(cloudUserFn, WorkerPropertyNames.PHASE, CombinePhase.ALL);
+      checkArgument(
+          phase.equals(CombinePhase.ALL) || phase.equals(CombinePhase.MERGE),
+          "Unexpected phase: %s",
+          phase);
+      if (phase.equals(CombinePhase.MERGE)) {
+        maybeMergingCombineFn = makeAppliedMergingFunction(combineFn);
+      } else {
+        maybeMergingCombineFn = combineFn;
+      }
+    }
+
+    StateInternalsFactory<?> stateInternalsFactory =
+        key -> executionContext.getStepContext(operationContext).stateInternals();
+
+    // This will be a GABW Fn for either batch or streaming, with combiner in it or not
+    GroupAlsoByWindowFn<?, ?> fn;
+
+    // This will be a FakeKeyedWorkItemCoder for streaming or null for batch
+    Coder<?> gabwInputCoder;
+
+    // TODO: do not do this with mess of "if"
+    if (isStreamingPipeline) {
+      if (maybeMergingCombineFn == null) {
+        fn =
+            StreamingGroupAlsoByWindowsDoFns.createForIterable(
+                windowingStrategy, stateInternalsFactory, ((KvCoder) kvCoder).getValueCoder());
+        gabwInputCoder = WindmillKeyedWorkItem.FakeKeyedWorkItemCoder.of(kvCoder);
+      } else {
+        fn =
+            StreamingGroupAlsoByWindowsDoFns.create(
+                windowingStrategy,
+                stateInternalsFactory,
+                (AppliedCombineFn) maybeMergingCombineFn,
+                ((KvCoder) kvCoder).getKeyCoder());
+        gabwInputCoder =
+            WindmillKeyedWorkItem.FakeKeyedWorkItemCoder.of(
+                ((AppliedCombineFn) maybeMergingCombineFn).getKvCoder());
+      }
+
+    } else {
+      if (maybeMergingCombineFn == null) {
+        fn =
+            BatchGroupAlsoByWindowsDoFns.createForIterable(
+                windowingStrategy, stateInternalsFactory, ((KvCoder) kvCoder).getValueCoder());
+        gabwInputCoder = null;
+      } else {
+        fn =
+            BatchGroupAlsoByWindowsDoFns.create(
+                windowingStrategy, (AppliedCombineFn) maybeMergingCombineFn);
+        gabwInputCoder = null;
+      }
+    }
+
+    // TODO: or anyhow related to it, do not do this with mess of "if"
+    if (maybeMergingCombineFn != null) {
+      return new GroupAlsoByWindowsParDoFn(
+          options,
+          fn,
+          windowingStrategy,
+          ((AppliedCombineFn) maybeMergingCombineFn).getSideInputViews(),
+          gabwInputCoder,
+          sideInputReader,
+          mainOutputTag,
+          executionContext.getStepContext(operationContext));
+    } else {
+      return new GroupAlsoByWindowsParDoFn(
+          options,
+          fn,
+          windowingStrategy,
+          null,
+          gabwInputCoder,
+          sideInputReader,
+          mainOutputTag,
+          executionContext.getStepContext(operationContext));
+    }
+  }
+
+  static WindowingStrategy deserializeWindowingStrategy(byte[] encodedWindowingStrategy)
+      throws InvalidProtocolBufferException {
+    RunnerApi.MessageWithComponents strategyProto =
+        RunnerApi.MessageWithComponents.parseFrom(encodedWindowingStrategy);
+    checkArgument(
+        strategyProto.getRootCase() == RootCase.WINDOWING_STRATEGY,
+        "Invalid windowing strategy: %s",
+        strategyProto);
+    return WindowingStrategyTranslation.fromProto(
+        strategyProto.getWindowingStrategy(),
+        RehydratedComponents.forComponents(strategyProto.getComponents()));
+  }
+
+  private static <K, AccumT>
+      AppliedCombineFn<K, AccumT, List<AccumT>, AccumT> makeAppliedMergingFunction(
+          AppliedCombineFn<K, ?, AccumT, ?> appliedFn) {
+    GlobalCombineFn<AccumT, List<AccumT>, AccumT> mergingCombineFn;
+    if (appliedFn.getFn() instanceof CombineFnWithContext) {
+      mergingCombineFn =
+          new MergingKeyedCombineFnWithContext<>(
+              (CombineFnWithContext<?, AccumT, ?>) appliedFn.getFn(),
+              appliedFn.getAccumulatorCoder());
+    } else {
+      mergingCombineFn =
+          new MergingCombineFn<>(
+              (CombineFn<?, AccumT, ?>) appliedFn.getFn(), appliedFn.getAccumulatorCoder());
+    }
+    return AppliedCombineFn.<K, AccumT, List<AccumT>, AccumT>withAccumulatorCoder(
+        mergingCombineFn,
+        ListCoder.of(appliedFn.getAccumulatorCoder()),
+        appliedFn.getSideInputViews(),
+        KvCoder.of(appliedFn.getKvCoder().getKeyCoder(), appliedFn.getAccumulatorCoder()),
+        appliedFn.getWindowingStrategy());
+  }
+
+  private static int MAX_ACCUMULATOR_BUFFER_SIZE = 10;
+
+  static class MergingCombineFn<K, AccumT> extends CombineFn<AccumT, List<AccumT>, AccumT> {
+
+    private final CombineFn<?, AccumT, ?> combineFn;
+    private final Coder<AccumT> accumCoder;
+
+    MergingCombineFn(CombineFn<?, AccumT, ?> combineFn, Coder<AccumT> accumCoder) {
+      this.combineFn = combineFn;
+      this.accumCoder = accumCoder;
+    }
+
+    @Override
+    public List<AccumT> createAccumulator() {
+      ArrayList<AccumT> result = new ArrayList<>();
+      result.add(this.combineFn.createAccumulator());
+      return result;
+    }
+
+    @Override
+    public List<AccumT> addInput(List<AccumT> accumulator, AccumT input) {
+      accumulator.add(input);
+      if (accumulator.size() < MAX_ACCUMULATOR_BUFFER_SIZE) {
+        return accumulator;
+      } else {
+        return mergeToSingleton(accumulator);
+      }
+    }
+
+    @Override
+    public List<AccumT> mergeAccumulators(Iterable<List<AccumT>> accumulators) {
+      return mergeToSingleton(Iterables.concat(accumulators));
+    }
+
+    @Override
+    public List<AccumT> compact(List<AccumT> accumulator) {
+      return mergeToSingleton(accumulator);
+    }
+
+    @Override
+    public AccumT extractOutput(List<AccumT> accumulator) {
+      if (accumulator.size() == 0) {
+        return combineFn.createAccumulator();
+      } else {
+        return combineFn.mergeAccumulators(accumulator);
+      }
+    }
+
+    private List<AccumT> mergeToSingleton(Iterable<AccumT> accumulators) {
+      List<AccumT> singleton = new ArrayList<>();
+      singleton.add(combineFn.mergeAccumulators(accumulators));
+      return singleton;
+    }
+
+    @Override
+    public Coder<List<AccumT>> getAccumulatorCoder(CoderRegistry registry, Coder<AccumT> inputCoder)
+        throws CannotProvideCoderException {
+      return ListCoder.of(accumCoder);
+    }
+  }
+
+  static class MergingKeyedCombineFnWithContext<K, AccumT>
+      extends CombineFnWithContext<AccumT, List<AccumT>, AccumT> {
+
+    private final CombineFnWithContext<?, AccumT, ?> combineFnWithContext;
+    private final Coder<AccumT> accumCoder;
+
+    MergingKeyedCombineFnWithContext(
+        CombineFnWithContext<?, AccumT, ?> combineFnWithContext, Coder<AccumT> accumCoder) {
+      this.combineFnWithContext = combineFnWithContext;
+      this.accumCoder = accumCoder;
+    }
+
+    @Override
+    public List<AccumT> createAccumulator(Context c) {
+      ArrayList<AccumT> result = new ArrayList<>();
+      result.add(this.combineFnWithContext.createAccumulator(c));
+      return result;
+    }
+
+    @Override
+    public List<AccumT> addInput(List<AccumT> accumulator, AccumT input, Context c) {
+      accumulator.add(input);
+      if (accumulator.size() < MAX_ACCUMULATOR_BUFFER_SIZE) {
+        return accumulator;
+      } else {
+        return mergeToSingleton(accumulator, c);
+      }
+    }
+
+    @Override
+    public List<AccumT> mergeAccumulators(Iterable<List<AccumT>> accumulators, Context c) {
+      return mergeToSingleton(Iterables.concat(accumulators), c);
+    }
+
+    @Override
+    public List<AccumT> compact(List<AccumT> accumulator, Context c) {
+      return mergeToSingleton(accumulator, c);
+    }
+
+    @Override
+    public AccumT extractOutput(List<AccumT> accumulator, Context c) {
+      if (accumulator.size() == 0) {
+        return combineFnWithContext.createAccumulator(c);
+      } else {
+        return combineFnWithContext.mergeAccumulators(accumulator, c);
+      }
+    }
+
+    private List<AccumT> mergeToSingleton(Iterable<AccumT> accumulators, Context c) {
... 85964 lines suppressed ...