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 ...